-
Notifications
You must be signed in to change notification settings - Fork 425
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
TEZ-4295: Could not decompress data. Buffer length is too small.
- Loading branch information
1 parent
0af54df
commit 7ffc783
Showing
3 changed files
with
189 additions
and
12 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
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
168 changes: 168 additions & 0 deletions
168
tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.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,168 @@ | ||
/** | ||
* 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.tez.runtime.library.utils; | ||
|
||
import java.io.IOException; | ||
import java.util.concurrent.CountDownLatch; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.Future; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.CommonConfigurationKeys; | ||
import org.apache.hadoop.io.compress.CodecPool; | ||
import org.apache.hadoop.io.compress.CompressionCodec; | ||
import org.apache.hadoop.io.compress.Compressor; | ||
import org.apache.hadoop.io.compress.Decompressor; | ||
import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; | ||
import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; | ||
import org.junit.Assert; | ||
import org.junit.Test; | ||
import org.mockito.Mockito; | ||
|
||
public class TestCodecUtils { | ||
|
||
@Test | ||
public void testConcurrentDecompressorCreationWithModifiedBuffersize() throws Exception { | ||
int modifiedBufferSize = 1000; | ||
int numberOfThreads = 1000; | ||
|
||
ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); | ||
|
||
Configuration conf = new Configuration(); | ||
conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); | ||
CompressionCodec codec = CodecUtils.getCodec(conf); | ||
|
||
Future<?>[] futures = new Future[numberOfThreads]; | ||
final CountDownLatch latch = new CountDownLatch(1); | ||
|
||
for (int i = 0; i < numberOfThreads; i++) { | ||
futures[i] = service.submit(() -> { | ||
try { | ||
waitForLatch(latch); | ||
|
||
Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
Decompressor decompressor = CodecUtils.getDecompressor(codec); | ||
CodecUtils.getDecompressedInputStreamWithBufferSize(codec, | ||
Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); | ||
|
||
Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
CodecPool.returnDecompressor(decompressor); | ||
} catch (IOException e) { | ||
throw new RuntimeException(e); | ||
} | ||
}); | ||
} | ||
latch.countDown(); | ||
|
||
for (Future<?> f : futures) { | ||
f.get(); | ||
} | ||
} | ||
|
||
@Test | ||
public void testConcurrentCompressorDecompressorCreation() throws Exception { | ||
int modifiedBufferSize = 1000; | ||
int numberOfThreads = 1000; | ||
|
||
ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); | ||
|
||
Configuration conf = new Configuration(); | ||
conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); | ||
CompressionCodec codec = CodecUtils.getCodec(conf); | ||
|
||
Future<?>[] futures = new Future[numberOfThreads]; | ||
final CountDownLatch latch = new CountDownLatch(1); | ||
|
||
for (int i = 0; i < numberOfThreads; i++) { | ||
// let's "randomly" choose from scenarios and test them concurrently | ||
// 1. getDecompressedInputStreamWithBufferSize | ||
if (i % 3 == 0) { | ||
futures[i] = service.submit(() -> { | ||
try { | ||
waitForLatch(latch); | ||
|
||
Assert.assertEquals( | ||
Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
Decompressor decompressor = CodecUtils.getDecompressor(codec); | ||
CodecUtils.getDecompressedInputStreamWithBufferSize(codec, | ||
Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); | ||
|
||
Assert.assertEquals( | ||
Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
CodecPool.returnDecompressor(decompressor); | ||
} catch (IOException e) { | ||
throw new RuntimeException(e); | ||
} | ||
}); | ||
// 2. getCompressor | ||
} else if (i % 3 == 1) { | ||
futures[i] = service.submit(() -> { | ||
waitForLatch(latch); | ||
|
||
Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
Compressor compressor = CodecUtils.getCompressor(codec); | ||
|
||
Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
CodecPool.returnCompressor(compressor); | ||
|
||
}); | ||
// 3. getDecompressor | ||
} else if (i % 3 == 2) { | ||
futures[i] = service.submit(() -> { | ||
waitForLatch(latch); | ||
|
||
Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
Decompressor decompressor = CodecUtils.getDecompressor(codec); | ||
|
||
Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), | ||
conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); | ||
|
||
CodecPool.returnDecompressor(decompressor); | ||
}); | ||
} | ||
} | ||
latch.countDown(); | ||
|
||
for (Future<?> f : futures) { | ||
f.get(); | ||
} | ||
} | ||
|
||
private void waitForLatch(CountDownLatch latch) { | ||
try { | ||
latch.await(); | ||
} catch (InterruptedException e) { | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
} |