forked from apache/hadoop
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
COMPX-12964: MAPREDUCE-7431. ShuffleHandler refactor and fix after Ne…
…tty4 upgrade. (apache#5311) This commit does not contain secrets Change-Id: Ic693413cba6aed4cea3b645c4b65c559450ec40d
- Loading branch information
1 parent
48f611e
commit 90158fa
Showing
11 changed files
with
2,056 additions
and
2,585 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
715 changes: 715 additions & 0 deletions
715
...apreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandler.java
Large diffs are not rendered by default.
Oops, something went wrong.
140 changes: 140 additions & 0 deletions
140
...e-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelHandlerContext.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,140 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hadoop.mapred; | ||
|
||
import io.netty.channel.group.ChannelGroup; | ||
|
||
import com.google.common.cache.LoadingCache; | ||
|
||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.io.ReadaheadPool; | ||
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; | ||
import org.apache.hadoop.util.Shell; | ||
|
||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_MAX_SHUFFLE_CONNECTIONS; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_BUFFER_SIZE; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MANAGE_OS_CACHE; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_READAHEAD_BYTES; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.MAX_SHUFFLE_CONNECTIONS; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_BUFFER_SIZE; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MANAGE_OS_CACHE; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_MAX_SESSION_OPEN_FILES; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_READAHEAD_BYTES; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SHUFFLE_TRANSFERTO_ALLOWED; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.SUFFLE_SSL_FILE_BUFFER_SIZE_KEY; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED; | ||
|
||
@SuppressWarnings("checkstyle:VisibilityModifier") | ||
public class ShuffleChannelHandlerContext { | ||
|
||
public final Configuration conf; | ||
public final JobTokenSecretManager secretManager; | ||
public final Map<String, String> userRsrc; | ||
public final LoadingCache<ShuffleHandler.AttemptPathIdentifier, | ||
ShuffleHandler.AttemptPathInfo> pathCache; | ||
public final IndexCache indexCache; | ||
public final ShuffleHandler.ShuffleMetrics metrics; | ||
public final ChannelGroup allChannels; | ||
|
||
|
||
public final boolean connectionKeepAliveEnabled; | ||
public final int sslFileBufferSize; | ||
public final int connectionKeepAliveTimeOut; | ||
public final int mapOutputMetaInfoCacheSize; | ||
|
||
public final AtomicInteger activeConnections = new AtomicInteger(); | ||
|
||
/** | ||
* Should the shuffle use posix_fadvise calls to manage the OS cache during | ||
* sendfile. | ||
*/ | ||
public final boolean manageOsCache; | ||
public final int readaheadLength; | ||
public final int maxShuffleConnections; | ||
public final int shuffleBufferSize; | ||
public final boolean shuffleTransferToAllowed; | ||
public final int maxSessionOpenFiles; | ||
public final ReadaheadPool readaheadPool = ReadaheadPool.getInstance(); | ||
|
||
public int port = -1; | ||
|
||
public ShuffleChannelHandlerContext(Configuration conf, | ||
Map<String, String> userRsrc, | ||
JobTokenSecretManager secretManager, | ||
LoadingCache<ShuffleHandler.AttemptPathIdentifier, | ||
ShuffleHandler.AttemptPathInfo> patCache, | ||
IndexCache indexCache, | ||
ShuffleHandler.ShuffleMetrics metrics, | ||
ChannelGroup allChannels) { | ||
this.conf = conf; | ||
this.userRsrc = userRsrc; | ||
this.secretManager = secretManager; | ||
this.pathCache = patCache; | ||
this.indexCache = indexCache; | ||
this.metrics = metrics; | ||
this.allChannels = allChannels; | ||
|
||
sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, | ||
DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE); | ||
connectionKeepAliveEnabled = | ||
conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, | ||
DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED); | ||
connectionKeepAliveTimeOut = | ||
Math.max(1, conf.getInt(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, | ||
DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT)); | ||
mapOutputMetaInfoCacheSize = | ||
Math.max(1, conf.getInt(SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE, | ||
DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE)); | ||
|
||
manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE, | ||
DEFAULT_SHUFFLE_MANAGE_OS_CACHE); | ||
|
||
readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES, | ||
DEFAULT_SHUFFLE_READAHEAD_BYTES); | ||
|
||
maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS, | ||
DEFAULT_MAX_SHUFFLE_CONNECTIONS); | ||
|
||
shuffleBufferSize = conf.getInt(SHUFFLE_BUFFER_SIZE, | ||
DEFAULT_SHUFFLE_BUFFER_SIZE); | ||
|
||
shuffleTransferToAllowed = conf.getBoolean(SHUFFLE_TRANSFERTO_ALLOWED, | ||
(Shell.WINDOWS)?WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED: | ||
DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED); | ||
|
||
maxSessionOpenFiles = conf.getInt(SHUFFLE_MAX_SESSION_OPEN_FILES, | ||
DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); | ||
} | ||
|
||
void setPort(int port) { | ||
this.port = port; | ||
} | ||
} |
74 changes: 74 additions & 0 deletions
74
...duce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleChannelInitializer.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hadoop.mapred; | ||
|
||
import io.netty.channel.ChannelInitializer; | ||
import io.netty.channel.ChannelPipeline; | ||
import io.netty.channel.socket.SocketChannel; | ||
import io.netty.handler.codec.http.HttpObjectAggregator; | ||
import io.netty.handler.codec.http.HttpServerCodec; | ||
import io.netty.handler.ssl.SslHandler; | ||
import io.netty.handler.stream.ChunkedWriteHandler; | ||
|
||
import java.io.IOException; | ||
import java.security.GeneralSecurityException; | ||
|
||
import org.apache.hadoop.security.ssl.SSLFactory; | ||
|
||
import static org.apache.hadoop.mapred.ShuffleHandler.TIMEOUT_HANDLER; | ||
import static org.apache.hadoop.mapred.ShuffleHandler.LOG; | ||
|
||
public class ShuffleChannelInitializer extends ChannelInitializer<SocketChannel> { | ||
|
||
public static final int MAX_CONTENT_LENGTH = 1 << 16; | ||
|
||
private final ShuffleChannelHandlerContext handlerContext; | ||
private final SSLFactory sslFactory; | ||
|
||
|
||
public ShuffleChannelInitializer(ShuffleChannelHandlerContext ctx, SSLFactory sslFactory) { | ||
this.handlerContext = ctx; | ||
this.sslFactory = sslFactory; | ||
} | ||
|
||
@Override | ||
public void initChannel(SocketChannel ch) throws GeneralSecurityException, IOException { | ||
LOG.debug("ShuffleChannelInitializer init; channel='{}'", ch.id()); | ||
|
||
ChannelPipeline pipeline = ch.pipeline(); | ||
if (sslFactory != null) { | ||
pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); | ||
} | ||
pipeline.addLast("http", new HttpServerCodec()); | ||
pipeline.addLast("aggregator", new HttpObjectAggregator(MAX_CONTENT_LENGTH)); | ||
pipeline.addLast("chunking", new ChunkedWriteHandler()); | ||
|
||
// An EventExecutorGroup could be specified to run in a | ||
// different thread than an I/O thread so that the I/O thread | ||
// is not blocked by a time-consuming task: | ||
// https://netty.io/4.1/api/io/netty/channel/ChannelPipeline.html | ||
pipeline.addLast("shuffle", new ShuffleChannelHandler(handlerContext)); | ||
|
||
pipeline.addLast(TIMEOUT_HANDLER, | ||
new ShuffleHandler.TimeoutHandler(handlerContext.connectionKeepAliveTimeOut)); | ||
// TODO factor security manager into pipeline | ||
// TODO factor out encode/decode to permit binary shuffle | ||
// TODO factor out decode of index to permit alt. models | ||
} | ||
} |
Oops, something went wrong.