You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2022/02/17 15:03:50 UTC

[GitHub] [drill] vdiravka commented on a change in pull request #2463: DRILL-8139: Data corruption and occasional segfaults querying Parquet/gzip under the async column reader and sync page reader

vdiravka commented on a change in pull request #2463:
URL: https://github.com/apache/drill/pull/2463#discussion_r809090421



##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/compression/AirliftBytesInputCompressor.java
##########
@@ -159,13 +160,13 @@ public void decompress(ByteBuffer input, int compressedSize, ByteBuffer output,
 
   @Override
   public void release() {
-    logger.debug(
-        "will release {} allocated buffers.",
-        this.allocatedBuffers.size()
-    );
+    int bufCount  = allocatedBuffers.size();
 
-    while (!this.allocatedBuffers.isEmpty()) {
-      this.allocator.release(allocatedBuffers.pop());
+    // LIFO release order to try to reduce memory fragmentation.
+    while (!allocatedBuffers.isEmpty()) {
+      allocator.release(allocatedBuffers.pop());
     }
+
+    logger.debug("released {} allocated buffers", bufCount);

Review comment:
       We possibly can assert or log the actual `allocatedBuffers.size()` to make sure allocated buffers are released

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/compression/DrillCompressionCodecFactory.java
##########
@@ -50,17 +52,31 @@
 
   // The set of codecs to be handled by aircompressor
   private static final Set<CompressionCodecName> AIRCOMPRESSOR_CODECS = new HashSet<>(
-      Arrays.asList(CompressionCodecName.LZ4, CompressionCodecName.LZO,
-          CompressionCodecName.SNAPPY, CompressionCodecName.ZSTD));
+      Arrays.asList(
+        CompressionCodecName.LZ4,
+        CompressionCodecName.LZO,
+        CompressionCodecName.SNAPPY,
+        CompressionCodecName.ZSTD
+      )
+  );
 
-  // pool of reused aircompressor compressors (parquet-mr's factory has its own)
+  // pool of reusable thread-safe aircompressor compressors (parquet-mr's factory

Review comment:
       ```suggestion
     // pool of reusable thread-safe aircompressor compressors (parquet-mr's factory has its own)
   ```
   

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/compression/DrillCompressionCodecFactory.java
##########
@@ -50,17 +52,31 @@
 
   // The set of codecs to be handled by aircompressor
   private static final Set<CompressionCodecName> AIRCOMPRESSOR_CODECS = new HashSet<>(
-      Arrays.asList(CompressionCodecName.LZ4, CompressionCodecName.LZO,
-          CompressionCodecName.SNAPPY, CompressionCodecName.ZSTD));
+      Arrays.asList(
+        CompressionCodecName.LZ4,
+        CompressionCodecName.LZO,
+        CompressionCodecName.SNAPPY,
+        CompressionCodecName.ZSTD
+      )
+  );
 
-  // pool of reused aircompressor compressors (parquet-mr's factory has its own)
+  // pool of reusable thread-safe aircompressor compressors (parquet-mr's factory
+  // has its own)
   private final Map<CompressionCodecName, AirliftBytesInputCompressor> airCompressors = new HashMap<>();
 
   // fallback parquet-mr compression codec factory
-  private CompressionCodecFactory parqCodecFactory;
+  private final CompressionCodecFactory parqCodecFactory;
 
   // direct memory allocator to be used during (de)compression
-  private ByteBufferAllocator allocator;
+  private final ByteBufferAllocator allocator;
+
+  // Start: members for working around a gzip concurrency bug c.f. DRILL-8139
+  private final Deque<CompressionCodecFactory> singleUseFactories;
+

Review comment:
       ```suggestion
   ```

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/compression/DrillCompressionCodecFactory.java
##########
@@ -80,38 +99,56 @@ public DrillCompressionCodecFactory(Configuration config, ByteBufferAllocator al
   }
 
   @Override
-  public BytesInputCompressor getCompressor(CompressionCodecName codecName) {
+  public synchronized BytesInputCompressor getCompressor(CompressionCodecName codecName) {
     if (AIRCOMPRESSOR_CODECS.contains(codecName)) {
       return airCompressors.computeIfAbsent(
           codecName,
           c -> new AirliftBytesInputCompressor(codecName, allocator)
       );
+    } else if (codecName != CompressionCodecName.SNAPPY) {
+      // Work around PARQUET-2126: construct a new codec factory every time to
+      // avoid a concurrrency bug c.f. DRILL-8139. Remove once PARQUET-2126 is
+      // fixed.  Snappy is immune because of the thread safety in the Xerial lib.
+      CompressionCodecFactory ccf = CodecFactory.createDirectCodecFactory(config, allocator, pageSize);
+      // hold onto a reference for later release()
+      singleUseFactories.add(ccf);
+      return ccf.getCompressor(codecName);
     } else {
       return parqCodecFactory.getCompressor(codecName);
     }
   }
 
   @Override
-  public BytesInputDecompressor getDecompressor(CompressionCodecName codecName) {
+  public synchronized BytesInputDecompressor getDecompressor(CompressionCodecName codecName) {
     if (AIRCOMPRESSOR_CODECS.contains(codecName)) {
       return airCompressors.computeIfAbsent(
           codecName,
           c -> new AirliftBytesInputCompressor(codecName, allocator)
       );
+    } else if (codecName != CompressionCodecName.SNAPPY) {

Review comment:
       Looks like existed `else` block starts to be never reached:
   When `codecName` is SNAPPY or other from `AIRCOMPRESSOR_CODECS` we will pass through the first condition 
   and for all other cases we will pass through the new `else if` block. If that by design the logic can be simplified: `else if` -> `else`. `return parqCodecFactory.getDecompressor(codecName);` possibly can be commented.
   
   The same for `#getCompressor` method

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/compression/DrillCompressionCodecFactory.java
##########
@@ -80,38 +99,56 @@ public DrillCompressionCodecFactory(Configuration config, ByteBufferAllocator al
   }
 
   @Override
-  public BytesInputCompressor getCompressor(CompressionCodecName codecName) {
+  public synchronized BytesInputCompressor getCompressor(CompressionCodecName codecName) {
     if (AIRCOMPRESSOR_CODECS.contains(codecName)) {
       return airCompressors.computeIfAbsent(
           codecName,
           c -> new AirliftBytesInputCompressor(codecName, allocator)
       );
+    } else if (codecName != CompressionCodecName.SNAPPY) {
+      // Work around PARQUET-2126: construct a new codec factory every time to
+      // avoid a concurrrency bug c.f. DRILL-8139. Remove once PARQUET-2126 is

Review comment:
       ```suggestion
        // avoid a concurrrency bug c.f. DRILL-8139. TODO: Remove once PARQUET-2126 is
   Write Preview
   ```

##########
File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/compression/DrillCompressionCodecFactory.java
##########
@@ -50,17 +52,31 @@
 
   // The set of codecs to be handled by aircompressor
   private static final Set<CompressionCodecName> AIRCOMPRESSOR_CODECS = new HashSet<>(
-      Arrays.asList(CompressionCodecName.LZ4, CompressionCodecName.LZO,
-          CompressionCodecName.SNAPPY, CompressionCodecName.ZSTD));
+      Arrays.asList(
+        CompressionCodecName.LZ4,
+        CompressionCodecName.LZO,
+        CompressionCodecName.SNAPPY,
+        CompressionCodecName.ZSTD
+      )
+  );
 
-  // pool of reused aircompressor compressors (parquet-mr's factory has its own)
+  // pool of reusable thread-safe aircompressor compressors (parquet-mr's factory
+  // has its own)
   private final Map<CompressionCodecName, AirliftBytesInputCompressor> airCompressors = new HashMap<>();
 
   // fallback parquet-mr compression codec factory
-  private CompressionCodecFactory parqCodecFactory;
+  private final CompressionCodecFactory parqCodecFactory;
 
   // direct memory allocator to be used during (de)compression
-  private ByteBufferAllocator allocator;
+  private final ByteBufferAllocator allocator;
+
+  // Start: members for working around a gzip concurrency bug c.f. DRILL-8139
+  private final Deque<CompressionCodecFactory> singleUseFactories;
+
+  private final Configuration config;
+

Review comment:
       ```suggestion
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org