-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26674][CORE]Consolidate CompositeByteBuf when reading large frame #23602
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 4 commits
0072d06
a2536da
9fd8ecd
9515621
1e3e9cf
96a71ed
f872e24
3fb7484
bc44188
ef63cdb
449efed
5f8c4eb
3aad18a
6ca6f71
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -43,6 +43,7 @@ public class TransportConf { | |
| private final String SPARK_NETWORK_IO_LAZYFD_KEY; | ||
| private final String SPARK_NETWORK_VERBOSE_METRICS; | ||
| private final String SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY; | ||
| private final String SPARK_NETWORK_IO_CONSOLIDATEBUFS_THRESHOLD_KEY; | ||
|
|
||
| private final ConfigProvider conf; | ||
|
|
||
|
|
@@ -66,6 +67,7 @@ public TransportConf(String module, ConfigProvider conf) { | |
| SPARK_NETWORK_IO_LAZYFD_KEY = getConfKey("io.lazyFD"); | ||
| SPARK_NETWORK_VERBOSE_METRICS = getConfKey("io.enableVerboseMetrics"); | ||
| SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY = getConfKey("io.enableTcpKeepAlive"); | ||
| SPARK_NETWORK_IO_CONSOLIDATEBUFS_THRESHOLD_KEY = getConfKey("io.consolidateBufsThreshold"); | ||
| } | ||
|
|
||
| public int getInt(String name, int defaultValue) { | ||
|
|
@@ -94,6 +96,23 @@ public boolean preferDirectBufs() { | |
| return conf.getBoolean(SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY, true); | ||
| } | ||
|
|
||
| /** The threshold for consolidation, it is derived upon the memoryOverhead in yarn mode. */ | ||
|
||
| public long consolidateBufsThreshold() { | ||
| boolean isDriver = conf.get("spark.executor.id").equals("driver"); | ||
liupc marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| final long MEMORY_OVERHEAD_MIN = 384L; | ||
| final double MEMORY_OVERHEAD_FACTOR = 0.1; | ||
| final double SHUFFLE_MEMORY_OVERHEAD_FACTOR = MEMORY_OVERHEAD_FACTOR * 0.6; | ||
| final double SHUFFLE_MEMORY_OVERHEAD_SAFE_FACTOR = SHUFFLE_MEMORY_OVERHEAD_FACTOR * 0.5; | ||
| long memory; | ||
| if (isDriver) { | ||
| memory = Math.max(JavaUtils.byteStringAsBytes(conf.get("spark.driver.memory")), MEMORY_OVERHEAD_MIN); | ||
| } else { | ||
| memory = Math.max(JavaUtils.byteStringAsBytes(conf.get("spark.executor.memory")), MEMORY_OVERHEAD_MIN); | ||
| } | ||
| long defaultConsolidateBufsThreshold = (long)(memory * SHUFFLE_MEMORY_OVERHEAD_SAFE_FACTOR); | ||
| return conf.getLong(SPARK_NETWORK_IO_CONSOLIDATEBUFS_THRESHOLD_KEY, defaultConsolidateBufsThreshold); | ||
| } | ||
|
|
||
| /** Connect timeout in milliseconds. Default 120 secs. */ | ||
| public int connectionTimeoutMs() { | ||
| long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -47,6 +47,67 @@ public void testFrameDecoding() throws Exception { | |
| verifyAndCloseDecoder(decoder, ctx, data); | ||
| } | ||
|
|
||
| @Test | ||
| public void testConsolidationForDecodingNonFullyWrittenByteBuf() { | ||
|
||
| TransportFrameDecoder decoder = new TransportFrameDecoder(); | ||
| ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); | ||
| List<ByteBuf> retained = new ArrayList<>(); | ||
| when(ctx.fireChannelRead(any())).thenAnswer(in -> { | ||
| ByteBuf buf = (ByteBuf) in.getArguments()[0]; | ||
| retained.add(buf); | ||
| return null; | ||
| }); | ||
| ByteBuf data1 = Unpooled.buffer(1024 * 1024); | ||
| data1.writeLong(1024 * 1024 + 8); | ||
| data1.writeByte(127); | ||
| ByteBuf data2 = Unpooled.buffer(1024 * 1024); | ||
| for (int i = 0; i < 1024 * 1024 - 1; i++) { | ||
| data2.writeByte(128); | ||
| } | ||
| int orignalCapacity = data1.capacity() + data2.capacity(); | ||
| try { | ||
| decoder.channelRead(ctx, data1); | ||
| decoder.channelRead(ctx, data2); | ||
| assertEquals(1, retained.size()); | ||
| assert(retained.get(0).capacity() < orignalCapacity); | ||
| } catch (Exception e) { | ||
liupc marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| release(data1); | ||
| release(data2); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testConsolidationPerf() { | ||
| TransportFrameDecoder decoder = new TransportFrameDecoder(300 * 1024 * 1024); | ||
| ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); | ||
| List<ByteBuf> retained = new ArrayList<>(); | ||
| when(ctx.fireChannelRead(any())).thenAnswer(in -> { | ||
| ByteBuf buf = (ByteBuf) in.getArguments()[0]; | ||
| retained.add(buf); | ||
| return null; | ||
| }); | ||
|
|
||
| ByteBuf buf = Unpooled.buffer(8); | ||
| try { | ||
| buf.writeLong(8 + 1024 * 1024 * 1000); | ||
| decoder.channelRead(ctx, buf); | ||
| for (int i = 0; i < 1000; i++) { | ||
| buf = Unpooled.buffer(1024 * 1024 * 2); | ||
| ByteBuf writtenBuf = Unpooled.buffer(1024 * 1024).writerIndex(1024 * 1024); | ||
| buf.writeBytes(writtenBuf); | ||
| writtenBuf.release(); | ||
| decoder.channelRead(ctx, buf); | ||
| } | ||
| assertEquals(1, retained.size()); | ||
| assertEquals(1024 * 1024 * 1000, retained.get(0).capacity()); | ||
| System.out.println("consolidated " + decoder.consolidatedCount + " times cost " + decoder.consolidatedTotalTime + " milis"); | ||
| } catch (Exception e) { | ||
| if (buf != null) { | ||
| release(buf); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testInterception() throws Exception { | ||
| int interceptedReads = 3; | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.