alexeykudinkin commented on code in PR #5497:
URL: https://github.com/apache/hudi/pull/5497#discussion_r929406153


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.hudi.io.storage;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Base class of Hudi's custom {@link ParquetWriter} implementations
+ *
+ * @param <R> target type of the object being written into Parquet files (for 
ex,
+ *           {@code IndexedRecord}, {@code InternalRow})
+ */
+public abstract class HoodieBaseParquetWriter<R> extends ParquetWriter<R> {
+
+  private static final int WRITTEN_RECORDS_THRESHOLD_FOR_FILE_SIZE_CHECK = 
1000;
+
+  private final AtomicLong writtenRecordCount = new AtomicLong(0);
+  private final long maxFileSize;
+  private long lastCachedDataSize = -1;
+
+  public HoodieBaseParquetWriter(Path file,
+                                 HoodieBaseParquetConfig<? extends 
WriteSupport<R>> parquetConfig) throws IOException {
+    super(HoodieWrapperFileSystem.convertToHoodiePath(file, 
parquetConfig.getHadoopConf()),
+        ParquetFileWriter.Mode.CREATE,
+        parquetConfig.getWriteSupport(),
+        parquetConfig.getCompressionCodecName(),
+        parquetConfig.getBlockSize(),
+        parquetConfig.getPageSize(),
+        parquetConfig.getPageSize(),
+        parquetConfig.dictionaryEnabled(),
+        DEFAULT_IS_VALIDATING_ENABLED,
+        DEFAULT_WRITER_VERSION,
+        FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
+
+    // We cannot accurately measure the snappy compressed output file size. We 
are choosing a
+    // conservative 10%
+    // TODO - compute this compression ratio dynamically by looking at the 
bytes written to the
+    // stream and the actual file size reported by HDFS
+    this.maxFileSize = parquetConfig.getMaxFileSize()
+        + Math.round(parquetConfig.getMaxFileSize() * 
parquetConfig.getCompressionRatio());
+  }
+
+  public boolean canWrite() {
+    // TODO we can actually do evaluation more accurately:

Review Comment:
   > +1 to the overall idea. but here is the deal - the size may not update 
until a row group is actually flushed out to storage. so getDataSize() simply 
returns 0 until then.
   
   It won't: it always returns accurate metric, b/c
   
   1. It keeps track how many bytes were written (`lastRowGroupEndPos`)
   2. It calculates the actual buffered footprint 
(`columnStore.getBufferedSize()`)
   
   With the second being the problem -- it always traverse all of the cached 
all groups to accurately calculate the in-memory footprint (and there's no 
internal caching). So what ended up happening it kept growing the buffer for 
the whole file (120Mb) not flushing in until closure which was making 
traversals quadratic in runtime.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.hudi.io.storage;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Base class of Hudi's custom {@link ParquetWriter} implementations
+ *
+ * @param <R> target type of the object being written into Parquet files (for 
ex,
+ *           {@code IndexedRecord}, {@code InternalRow})
+ */
+public abstract class HoodieBaseParquetWriter<R> extends ParquetWriter<R> {
+
+  private static final int WRITTEN_RECORDS_THRESHOLD_FOR_FILE_SIZE_CHECK = 
1000;
+
+  private final AtomicLong writtenRecordCount = new AtomicLong(0);
+  private final long maxFileSize;
+  private long lastCachedDataSize = -1;
+
+  public HoodieBaseParquetWriter(Path file,
+                                 HoodieBaseParquetConfig<? extends 
WriteSupport<R>> parquetConfig) throws IOException {
+    super(HoodieWrapperFileSystem.convertToHoodiePath(file, 
parquetConfig.getHadoopConf()),
+        ParquetFileWriter.Mode.CREATE,
+        parquetConfig.getWriteSupport(),
+        parquetConfig.getCompressionCodecName(),
+        parquetConfig.getBlockSize(),
+        parquetConfig.getPageSize(),
+        parquetConfig.getPageSize(),
+        parquetConfig.dictionaryEnabled(),
+        DEFAULT_IS_VALIDATING_ENABLED,
+        DEFAULT_WRITER_VERSION,
+        FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
+
+    // We cannot accurately measure the snappy compressed output file size. We 
are choosing a
+    // conservative 10%
+    // TODO - compute this compression ratio dynamically by looking at the 
bytes written to the
+    // stream and the actual file size reported by HDFS
+    this.maxFileSize = parquetConfig.getMaxFileSize()
+        + Math.round(parquetConfig.getMaxFileSize() * 
parquetConfig.getCompressionRatio());
+  }
+
+  public boolean canWrite() {
+    // TODO we can actually do evaluation more accurately:
+    //      if we cache last data size check, since we account for how many 
records

Review Comment:
   I validated that `getDataSize` returns accurate results for buffered data



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to