|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.hadoop.mapred; |
| 20 | + |
| 21 | +import io.netty.channel.group.ChannelGroup; |
| 22 | + |
| 23 | +import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache; |
| 24 | + |
| 25 | +import java.util.Map; |
| 26 | +import java.util.concurrent.atomic.AtomicInteger; |
| 27 | + |
| 28 | +import org.apache.hadoop.conf.Configuration; |
| 29 | +import org.apache.hadoop.io.ReadaheadPool; |
| 30 | +import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; |
| 31 | +import org.apache.hadoop.util.Shell; |
| 32 | + |
| 33 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_MAX_SHUFFLE_CONNECTIONS; |
| 34 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_BUFFER_SIZE; |
| 35 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED; |
| 36 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT; |
| 37 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MANAGE_OS_CACHE; |
| 38 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE; |
| 39 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES; |
| 40 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_READAHEAD_BYTES; |
| 41 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED; |
| 42 | +import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE; |
| 43 | +import static org.apache.hadoop.mapred.ShuffleHandler.MAX_SHUFFLE_CONNECTIONS; |
| 44 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_BUFFER_SIZE; |
| 45 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED; |
| 46 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT; |
| 47 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MANAGE_OS_CACHE; |
| 48 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE; |
| 49 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MAX_SESSION_OPEN_FILES; |
| 50 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_READAHEAD_BYTES; |
| 51 | +import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_TRANSFERTO_ALLOWED; |
| 52 | +import static org.apache.hadoop.mapred.ShuffleHandler.SUFFLE_SSL_FILE_BUFFER_SIZE_KEY; |
| 53 | +import static org.apache.hadoop.mapred.ShuffleHandler.WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED; |
| 54 | + |
| 55 | +@SuppressWarnings("checkstyle:VisibilityModifier") |
| 56 | +public class ShuffleChannelHandlerContext { |
| 57 | + |
| 58 | + public final Configuration conf; |
| 59 | + public final JobTokenSecretManager secretManager; |
| 60 | + public final Map<String, String> userRsrc; |
| 61 | + public final LoadingCache<ShuffleHandler.AttemptPathIdentifier, |
| 62 | + ShuffleHandler.AttemptPathInfo> pathCache; |
| 63 | + public final IndexCache indexCache; |
| 64 | + public final ShuffleHandler.ShuffleMetrics metrics; |
| 65 | + public final ChannelGroup allChannels; |
| 66 | + |
| 67 | + |
| 68 | + public final boolean connectionKeepAliveEnabled; |
| 69 | + public final int sslFileBufferSize; |
| 70 | + public final int connectionKeepAliveTimeOut; |
| 71 | + public final int mapOutputMetaInfoCacheSize; |
| 72 | + |
| 73 | + public final AtomicInteger activeConnections = new AtomicInteger(); |
| 74 | + |
| 75 | + /** |
| 76 | + * Should the shuffle use posix_fadvise calls to manage the OS cache during |
| 77 | + * sendfile. |
| 78 | + */ |
| 79 | + public final boolean manageOsCache; |
| 80 | + public final int readaheadLength; |
| 81 | + public final int maxShuffleConnections; |
| 82 | + public final int shuffleBufferSize; |
| 83 | + public final boolean shuffleTransferToAllowed; |
| 84 | + public final int maxSessionOpenFiles; |
| 85 | + public final ReadaheadPool readaheadPool = ReadaheadPool.getInstance(); |
| 86 | + |
| 87 | + public int port = -1; |
| 88 | + |
| 89 | + public ShuffleChannelHandlerContext(Configuration conf, |
| 90 | + Map<String, String> userRsrc, |
| 91 | + JobTokenSecretManager secretManager, |
| 92 | + LoadingCache<ShuffleHandler.AttemptPathIdentifier, |
| 93 | + ShuffleHandler.AttemptPathInfo> patCache, |
| 94 | + IndexCache indexCache, |
| 95 | + ShuffleHandler.ShuffleMetrics metrics, |
| 96 | + ChannelGroup allChannels) { |
| 97 | + this.conf = conf; |
| 98 | + this.userRsrc = userRsrc; |
| 99 | + this.secretManager = secretManager; |
| 100 | + this.pathCache = patCache; |
| 101 | + this.indexCache = indexCache; |
| 102 | + this.metrics = metrics; |
| 103 | + this.allChannels = allChannels; |
| 104 | + |
| 105 | + sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, |
| 106 | + DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE); |
| 107 | + connectionKeepAliveEnabled = |
| 108 | + conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, |
| 109 | + DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED); |
| 110 | + connectionKeepAliveTimeOut = |
| 111 | + Math.max(1, conf.getInt(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, |
| 112 | + DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT)); |
| 113 | + mapOutputMetaInfoCacheSize = |
| 114 | + Math.max(1, conf.getInt(SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE, |
| 115 | + DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE)); |
| 116 | + |
| 117 | + manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE, |
| 118 | + DEFAULT_SHUFFLE_MANAGE_OS_CACHE); |
| 119 | + |
| 120 | + readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES, |
| 121 | + DEFAULT_SHUFFLE_READAHEAD_BYTES); |
| 122 | + |
| 123 | + maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS, |
| 124 | + DEFAULT_MAX_SHUFFLE_CONNECTIONS); |
| 125 | + |
| 126 | + shuffleBufferSize = conf.getInt(SHUFFLE_BUFFER_SIZE, |
| 127 | + DEFAULT_SHUFFLE_BUFFER_SIZE); |
| 128 | + |
| 129 | + shuffleTransferToAllowed = conf.getBoolean(SHUFFLE_TRANSFERTO_ALLOWED, |
| 130 | + (Shell.WINDOWS)?WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED: |
| 131 | + DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED); |
| 132 | + |
| 133 | + maxSessionOpenFiles = conf.getInt(SHUFFLE_MAX_SESSION_OPEN_FILES, |
| 134 | + DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); |
| 135 | + } |
| 136 | + |
| 137 | + void setPort(int port) { |
| 138 | + this.port = port; |
| 139 | + } |
| 140 | +} |
0 commit comments