|
| 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 | +package org.apache.tez.runtime.library.utils; |
| 19 | + |
| 20 | +import java.io.IOException; |
| 21 | +import java.util.concurrent.CountDownLatch; |
| 22 | +import java.util.concurrent.ExecutorService; |
| 23 | +import java.util.concurrent.Executors; |
| 24 | +import java.util.concurrent.Future; |
| 25 | + |
| 26 | +import org.apache.hadoop.conf.Configuration; |
| 27 | +import org.apache.hadoop.fs.CommonConfigurationKeys; |
| 28 | +import org.apache.hadoop.io.compress.CodecPool; |
| 29 | +import org.apache.hadoop.io.compress.CompressionCodec; |
| 30 | +import org.apache.hadoop.io.compress.Compressor; |
| 31 | +import org.apache.hadoop.io.compress.Decompressor; |
| 32 | +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; |
| 33 | +import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; |
| 34 | +import org.junit.Assert; |
| 35 | +import org.junit.Test; |
| 36 | +import org.mockito.Mockito; |
| 37 | + |
| 38 | +public class TestCodecUtils { |
| 39 | + |
| 40 | + @Test |
| 41 | + public void testConcurrentDecompressorCreationWithModifiedBuffersize() throws Exception { |
| 42 | + int modifiedBufferSize = 1000; |
| 43 | + int numberOfThreads = 1000; |
| 44 | + |
| 45 | + ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); |
| 46 | + |
| 47 | + Configuration conf = new Configuration(); |
| 48 | + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); |
| 49 | + CompressionCodec codec = CodecUtils.getCodec(conf); |
| 50 | + |
| 51 | + Future<?>[] futures = new Future[numberOfThreads]; |
| 52 | + final CountDownLatch latch = new CountDownLatch(1); |
| 53 | + |
| 54 | + for (int i = 0; i < numberOfThreads; i++) { |
| 55 | + futures[i] = service.submit(() -> { |
| 56 | + try { |
| 57 | + waitForLatch(latch); |
| 58 | + |
| 59 | + Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 60 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 61 | + |
| 62 | + Decompressor decompressor = CodecUtils.getDecompressor(codec); |
| 63 | + CodecUtils.getDecompressedInputStreamWithBufferSize(codec, |
| 64 | + Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); |
| 65 | + |
| 66 | + Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 67 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 68 | + |
| 69 | + CodecPool.returnDecompressor(decompressor); |
| 70 | + } catch (IOException e) { |
| 71 | + throw new RuntimeException(e); |
| 72 | + } |
| 73 | + }); |
| 74 | + } |
| 75 | + latch.countDown(); |
| 76 | + |
| 77 | + for (Future<?> f : futures) { |
| 78 | + f.get(); |
| 79 | + } |
| 80 | + } |
| 81 | + |
| 82 | + @Test |
| 83 | + public void testConcurrentCompressorDecompressorCreation() throws Exception { |
| 84 | + int modifiedBufferSize = 1000; |
| 85 | + int numberOfThreads = 1000; |
| 86 | + |
| 87 | + ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); |
| 88 | + |
| 89 | + Configuration conf = new Configuration(); |
| 90 | + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); |
| 91 | + CompressionCodec codec = CodecUtils.getCodec(conf); |
| 92 | + |
| 93 | + Future<?>[] futures = new Future[numberOfThreads]; |
| 94 | + final CountDownLatch latch = new CountDownLatch(1); |
| 95 | + |
| 96 | + for (int i = 0; i < numberOfThreads; i++) { |
| 97 | + // let's "randomly" choose from scenarios and test them concurrently |
| 98 | + // 1. getDecompressedInputStreamWithBufferSize |
| 99 | + if (i % 3 == 0) { |
| 100 | + futures[i] = service.submit(() -> { |
| 101 | + try { |
| 102 | + waitForLatch(latch); |
| 103 | + |
| 104 | + Assert.assertEquals( |
| 105 | + Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 106 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 107 | + |
| 108 | + Decompressor decompressor = CodecUtils.getDecompressor(codec); |
| 109 | + CodecUtils.getDecompressedInputStreamWithBufferSize(codec, |
| 110 | + Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); |
| 111 | + |
| 112 | + Assert.assertEquals( |
| 113 | + Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 114 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 115 | + |
| 116 | + CodecPool.returnDecompressor(decompressor); |
| 117 | + } catch (IOException e) { |
| 118 | + throw new RuntimeException(e); |
| 119 | + } |
| 120 | + }); |
| 121 | + // 2. getCompressor |
| 122 | + } else if (i % 3 == 1) { |
| 123 | + futures[i] = service.submit(() -> { |
| 124 | + waitForLatch(latch); |
| 125 | + |
| 126 | + Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 127 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 128 | + |
| 129 | + Compressor compressor = CodecUtils.getCompressor(codec); |
| 130 | + |
| 131 | + Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 132 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 133 | + |
| 134 | + CodecPool.returnCompressor(compressor); |
| 135 | + |
| 136 | + }); |
| 137 | + // 3. getDecompressor |
| 138 | + } else if (i % 3 == 2) { |
| 139 | + futures[i] = service.submit(() -> { |
| 140 | + waitForLatch(latch); |
| 141 | + |
| 142 | + Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 143 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 144 | + |
| 145 | + Decompressor decompressor = CodecUtils.getDecompressor(codec); |
| 146 | + |
| 147 | + Assert.assertEquals(Integer.toString(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_DEFAULT), |
| 148 | + conf.get(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY)); |
| 149 | + |
| 150 | + CodecPool.returnDecompressor(decompressor); |
| 151 | + }); |
| 152 | + } |
| 153 | + } |
| 154 | + latch.countDown(); |
| 155 | + |
| 156 | + for (Future<?> f : futures) { |
| 157 | + f.get(); |
| 158 | + } |
| 159 | + } |
| 160 | + |
| 161 | + private void waitForLatch(CountDownLatch latch) { |
| 162 | + try { |
| 163 | + latch.await(); |
| 164 | + } catch (InterruptedException e) { |
| 165 | + throw new RuntimeException(e); |
| 166 | + } |
| 167 | + } |
| 168 | +} |
0 commit comments