vinothchandar commented on code in PR #13365:
URL: https://github.com/apache/hudi/pull/13365#discussion_r2122348417


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateRewriteHandle.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.rewrite.HoodieFileMetadataMerger;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriter;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriterFactory;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+public class HoodieCreateRewriteHandle<T, I, K, O> extends 
HoodieCreateHandle<T, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieCreateRewriteHandle.class);
+
+  protected final HoodieFileRewriter rewriter;
+
+  private final List<StoragePath> inputFiles;
+
+  public HoodieCreateRewriteHandle(
+      HoodieWriteConfig config,
+      String instantTime,
+      String partitionPath,
+      String fileId,
+      HoodieTable<T, I, K, O> hoodieTable,
+      TaskContextSupplier taskContextSupplier,
+      List<StoragePath> inputFilePaths,
+      boolean preserveMetadata) {
+    super(
+        config,
+        instantTime,
+        hoodieTable,
+        partitionPath,
+        fileId,
+        Option.empty(),
+        taskContextSupplier,
+        preserveMetadata,
+        false);
+    try {
+      this.inputFiles = inputFilePaths;
+      HoodieFileMetadataMerger fileMetadataMerger = new 
HoodieFileMetadataMerger();
+      this.rewriter = HoodieFileRewriterFactory.getFileRewriter(
+          inputFilePaths,
+          path,
+          hoodieTable.getStorageConf().unwrapAs(Configuration.class),
+          hoodieTable.getConfig(),
+          fileMetadataMerger,
+          config.getRecordMerger().getRecordType(), 
this.writeSchemaWithMetaFields);
+    } catch (IOException e) {
+      LOG.error("Fail to create file rewriter, cause: ", e);
+      throw new HoodieException(e);
+    }
+  }
+
+  public void rewrite() {
+    LOG.info("Start to rewrite source files " + this.inputFiles + " into 
target file: " + this.path);
+    long start = System.currentTimeMillis();
+    long records = 0;
+    try {
+      records = this.rewriter.rewrite();
+    } catch (IOException e) {
+      throw new HoodieIOException(e.getMessage(), e);
+    } finally {
+      this.recordsWritten = records;
+      this.insertRecordsWritten = records;
+    }
+    LOG.info("Finish rewriting " + this.path + ". Using " + 
(System.currentTimeMillis() - start) + " mills");
+  }
+
+  @Override
+  public List<WriteStatus> close() {
+    try {
+      this.rewriter.close();
+      return super.close();
+    } catch (IOException e) {
+      LOG.error("Fail to close the rewrite handle for path: " + path);
+      throw new HoodieIOException(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public boolean canWrite(HoodieRecord record) {

Review Comment:
   So we don't do any file sizing here? we need logger.warn(..) etc noting that 
this change of behavior. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateRewriteHandle.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.rewrite.HoodieFileMetadataMerger;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriter;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriterFactory;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+public class HoodieCreateRewriteHandle<T, I, K, O> extends 
HoodieCreateHandle<T, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieCreateRewriteHandle.class);
+
+  protected final HoodieFileRewriter rewriter;
+
+  private final List<StoragePath> inputFiles;
+
+  public HoodieCreateRewriteHandle(
+      HoodieWriteConfig config,
+      String instantTime,
+      String partitionPath,
+      String fileId,
+      HoodieTable<T, I, K, O> hoodieTable,
+      TaskContextSupplier taskContextSupplier,
+      List<StoragePath> inputFilePaths,
+      boolean preserveMetadata) {
+    super(
+        config,
+        instantTime,
+        hoodieTable,
+        partitionPath,
+        fileId,
+        Option.empty(),
+        taskContextSupplier,
+        preserveMetadata,
+        false);
+    try {
+      this.inputFiles = inputFilePaths;
+      HoodieFileMetadataMerger fileMetadataMerger = new 
HoodieFileMetadataMerger();
+      this.rewriter = HoodieFileRewriterFactory.getFileRewriter(
+          inputFilePaths,
+          path,
+          hoodieTable.getStorageConf().unwrapAs(Configuration.class),
+          hoodieTable.getConfig(),
+          fileMetadataMerger,
+          config.getRecordMerger().getRecordType(), 
this.writeSchemaWithMetaFields);
+    } catch (IOException e) {
+      LOG.error("Fail to create file rewriter, cause: ", e);
+      throw new HoodieException(e);
+    }
+  }
+
+  public void rewrite() {
+    LOG.info("Start to rewrite source files " + this.inputFiles + " into 
target file: " + this.path);
+    long start = System.currentTimeMillis();

Review Comment:
   use HoodieTimer
   



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/parquet/io/TestHoodieParquetFileRewriter.java:
##########
@@ -0,0 +1,459 @@
+/*
+ * 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.parquet.io;
+
+import org.apache.hudi.io.storage.rewrite.HoodieFileMetadataMerger;
+import org.apache.hudi.storage.StoragePath;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.Version;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+import static org.apache.parquet.schema.Type.Repetition.REPEATED;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class TestHoodieParquetFileRewriter {
+
+  private final int numRecord = 1;
+  private Configuration conf = new Configuration();
+  private List<TestFile> inputFiles = null;
+  private String outputFile = null;
+  private HoodieParquetFileRewriter rewriter = null;
+
+  @BeforeEach
+  public void setUp() {
+    outputFile = TestFileBuilder.createTempFile("test");
+  }
+
+  @AfterEach
+  public void after() {
+    if (outputFile != null) {
+      TestFileBuilder.deleteTempFile(outputFile);
+    }
+    if (inputFiles != null) {
+      
inputFiles.stream().map(TestFile::getFileName).forEach(TestFileBuilder::deleteTempFile);
+    }
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    MessageType schema = createSchema();
+    inputFiles = new ArrayList<>();
+    inputFiles.add(makeTestFile(schema, "GZIP"));
+    inputFiles.add(makeTestFile(schema, "GZIP"));
+
+    rewriter = parquetFileRewriter(schema, "GZIP");
+    rewriter.rewrite();
+    rewriter.close();
+
+    // Verify the schema are not changed
+    ParquetMetadata pmd = ParquetFileReader.readFooter(conf, new 
Path(outputFile), ParquetMetadataConverter.NO_FILTER);
+    MessageType fileSchema = pmd.getFileMetaData().getSchema();
+    assertEquals(schema, fileSchema);
+    validateSchema(fileSchema);
+
+    // Verify codec
+    verifyCodec(outputFile, CompressionCodecName.GZIP);
+
+    // Verify the merged data are not changed
+    validateColumnData();
+
+    // Verify the page index
+    validatePageIndex(0, 1, 2, 3, 4);
+
+    // Verify original.created.by is preserved
+    validateCreatedBy();
+  }
+
+  @Test
+  public void testTranslateCodec() throws Exception {
+    MessageType schema = createSchema();
+    inputFiles = new ArrayList<>();
+    inputFiles.add(makeTestFile(schema, "GZIP"));
+    inputFiles.add(makeTestFile(schema, "UNCOMPRESSED"));
+
+    rewriter = parquetFileRewriter(schema, "ZSTD");
+    rewriter.rewrite();
+    rewriter.close();
+
+    // Verify the schema are not changed for the columns not pruned
+    ParquetMetadata pmd = ParquetFileReader.readFooter(conf, new 
Path(outputFile), ParquetMetadataConverter.NO_FILTER);
+    MessageType fileSchema = pmd.getFileMetaData().getSchema();
+    assertEquals(schema, fileSchema);
+    validateSchema(fileSchema);
+
+    // Verify codec has been translated
+    verifyCodec(outputFile, CompressionCodecName.ZSTD);
+
+    // Verify the data are not changed for the columns not pruned
+    validateColumnData();
+
+    // Verify the page index
+    validatePageIndex(0, 1, 2, 3, 4);
+
+    // Verify original.created.by is preserved
+    validateCreatedBy();
+  }
+
+  @Test
+  public void testDifferentSchema() throws Exception {
+    MessageType schema1 = new MessageType("schema",
+        new PrimitiveType(OPTIONAL, INT64, "DocId"),
+        new PrimitiveType(REQUIRED, BINARY, "Name"),
+        new PrimitiveType(OPTIONAL, BINARY, "Gender"),
+        new GroupType(OPTIONAL, "Links",
+            new PrimitiveType(REPEATED, BINARY, "Backward"),
+            new PrimitiveType(REPEATED, BINARY, "Forward")));
+    MessageType schema2 = new MessageType("schema",
+        new PrimitiveType(OPTIONAL, INT64, "DocId"),
+        new PrimitiveType(REQUIRED, BINARY, "Name"),
+        new PrimitiveType(OPTIONAL, BINARY, "Gender"));
+    inputFiles = new ArrayList<>();
+    inputFiles.add(makeTestFile(schema1, "UNCOMPRESSED"));
+    inputFiles.add(makeTestFile(schema2, "UNCOMPRESSED"));
+
+    rewriter = parquetFileRewriter(schema1, "UNCOMPRESSED");
+    rewriter.rewrite();
+    rewriter.close();
+
+    // Verify the schema are not changed for the columns not pruned
+    ParquetMetadata pmd = ParquetFileReader.readFooter(conf, new 
Path(outputFile), ParquetMetadataConverter.NO_FILTER);
+    MessageType schema = pmd.getFileMetaData().getSchema();
+    validateSchema(schema);
+
+    // Verify codec has been translated
+    verifyCodec(outputFile, CompressionCodecName.UNCOMPRESSED);
+
+    // Verify the data are not changed
+    validateColumnData();
+
+    // Verify the page index
+    validatePageIndex(0, 1, 2);
+
+    // Verify original.created.by is preserved
+    validateCreatedBy();
+  }
+
+  @Test
+  public void testHoodieMetaColumn() throws Exception {
+    MessageType schema = new MessageType("schema",
+        new PrimitiveType(OPTIONAL, BINARY, FILENAME_METADATA_FIELD),
+        new PrimitiveType(OPTIONAL, INT64, "DocId"),
+        new PrimitiveType(REQUIRED, BINARY, "Name"),
+        new PrimitiveType(OPTIONAL, BINARY, "Gender"),
+        new GroupType(OPTIONAL, "Links",
+            new PrimitiveType(REPEATED, BINARY, "Backward"),
+            new PrimitiveType(REPEATED, BINARY, "Forward")));
+    inputFiles = new ArrayList<>();
+    inputFiles.add(makeTestFile(schema, "GZIP"));
+    inputFiles.add(makeTestFile(schema, "GZIP"));
+
+    rewriter = parquetFileRewriter(schema, "GZIP");
+    rewriter.rewrite();
+    rewriter.close();
+
+    // Verify the schema are not changed for the columns not pruned
+    ParquetMetadata pmd = ParquetFileReader.readFooter(conf, new 
Path(outputFile), ParquetMetadataConverter.NO_FILTER);
+    MessageType fileSchema = pmd.getFileMetaData().getSchema();
+    assertEquals(schema, fileSchema);
+
+    // Verify codec has been translated
+    verifyCodec(outputFile, CompressionCodecName.GZIP);
+
+    // Verify the data are not changed
+    validateColumnData();

Review Comment:
   would n't some meta fields like `_hoodie_file_name` change.
   



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/SingleFileHandleRewriteFactory.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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;
+
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Rewrite all inputFilePaths related files into one huge file.

Review Comment:
   "rewrite" is lil confusing, since no data is changing.. 
   
   "Combine all input file paths quickly into a single large file" ?
   



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java:
##########
@@ -103,9 +103,9 @@ public HoodieCreateHandle(HoodieWriteConfig config, String 
instantTime, HoodieTa
       partitionMetadata.trySave();
       createMarkerFile(partitionPath,
           FSUtils.makeBaseFileName(this.instantTime, this.writeToken, 
this.fileId, hoodieTable.getBaseFileExtension()));
-      this.fileWriter =
-          HoodieFileWriterFactory.getFileWriter(instantTime, path, 
hoodieTable.getStorage(), config,
-              writeSchemaWithMetaFields, this.taskContextSupplier, 
config.getRecordMerger().getRecordType());
+      this.fileWriter = initWriter

Review Comment:
   not a fan of using `null` .. can we instead make the overall `fileWriter` 
initialization lazy?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateRewriteHandle.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.rewrite.HoodieFileMetadataMerger;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriter;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriterFactory;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+

Review Comment:
   javadocs



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/parquet/io/TestHoodieParquetFileRewriter.java:
##########
@@ -0,0 +1,459 @@
+/*
+ * 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.parquet.io;
+
+import org.apache.hudi.io.storage.rewrite.HoodieFileMetadataMerger;
+import org.apache.hudi.storage.StoragePath;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.Version;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.DataPageHeader;
+import org.apache.parquet.format.DataPageHeaderV2;
+import org.apache.parquet.format.PageHeader;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+import static org.apache.parquet.schema.Type.Repetition.REPEATED;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class TestHoodieParquetFileRewriter {

Review Comment:
   can we add more tests from the hudi context.. covering meta fields, stats 
and so forth. bloom filters.. 



##########
hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java:
##########
@@ -125,5 +126,42 @@ private void 
extractAndSetInternalBloomFilter(DataInputStream dis) throws IOExce
     internalDynamicBloomFilter = new InternalDynamicBloomFilter();
     internalDynamicBloomFilter.readFields(dis);
   }
+
+  @Override
+  public void or(BloomFilter other) {

Review Comment:
   unit tests



##########
hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java:
##########
@@ -54,4 +54,10 @@ public interface BloomFilter {
    * @return the bloom index type code.
    **/
   BloomFilterTypeCode getBloomFilterTypeCode();
+
+  /**
+   * Performs a logical OR operations with other BloomFilter.
+   * @param other
+   */
+  void or(BloomFilter other);

Review Comment:
   unit tests. 
   



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/rewrite/HoodieFileRewriterFactory.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.storage.StoragePath;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
+
+public class HoodieFileRewriterFactory {
+
+  private static HoodieFileRewriterFactory getWriterFactory(HoodieRecordType 
recordType, String extension) {
+    if (PARQUET.getFileExtension().equals(extension)) {
+      try {
+        Class<?> clazz = 
ReflectionUtils.getClass("org.apache.hudi.parquet.io.HoodieParquetRewriterFactory");
+        return (HoodieFileRewriterFactory) clazz.newInstance();
+      } catch (IllegalAccessException | IllegalArgumentException | 
InstantiationException e) {
+        throw new HoodieException("Unable to create hoodie avro parquet file 
writer factory", e);
+      }
+    }
+    throw new UnsupportedOperationException(extension + " file format not 
supported yet.");
+  }
+
+  public static <T, I, K, O> HoodieFileRewriter getFileRewriter(
+      List<StoragePath> inputFilePaths,
+      StoragePath targetFilePath,
+      Configuration conf,
+      HoodieConfig config,
+      HoodieFileMetadataMerger metadataMerger,
+      HoodieRecordType recordType,
+      Schema writeSchemaWithMetaFields) throws IOException {
+    String extension = FSUtils.getFileExtension(targetFilePath.getName());
+    HoodieFileRewriterFactory factory = getWriterFactory(recordType, 
extension);
+    return factory.newFileRewriter(inputFilePaths, targetFilePath, conf, 
config, metadataMerger, writeSchemaWithMetaFields);
+  }
+
+  protected <T> HoodieFileRewriter newFileRewriter(

Review Comment:
   can we avoid this



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkStreamCopyClusteringExecutionStrategy.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.client.clustering.run.strategy;
+
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.ClusteringGroupInfo;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.io.HoodieCreateRewriteHandle;
+import org.apache.hudi.io.SingleFileHandleRewriteFactory;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+import org.apache.avro.Schema;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE;
+
+/**
+ * Clustering strategy to submit single spark jobs using streaming copy
+ * PAY ATTENTION!!!
+ * IN THIS STRATEGY
+ *  1. Only support clustering for cow table.
+ *  2. Sort function is not supported yet.
+ *  3. Each clustering group only has one task to write.
+ */
+public class SparkStreamCopyClusteringExecutionStrategy<T> extends 
SparkSortAndSizeExecutionStrategy<T> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkStreamCopyClusteringExecutionStrategy.class);
+
+  public SparkStreamCopyClusteringExecutionStrategy(

Review Comment:
   lets please add UTs for this class. and all the new files introduced



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/rewrite/HoodieFileMetadataMerger.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.util.StringUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static 
org.apache.hudi.avro.HoodieBloomFilterWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
+
+public class HoodieFileMetadataMerger {
+
+  private final HashMap<String, String> mergedMateData = new HashMap<>();
+
+  private BloomFilter bloomFilter;
+
+  private String minRecordKey;
+
+  private String maxRecordKey;
+
+  public HoodieFileMetadataMerger() {
+
+  }
+
+  public Map<String, String> mergeMetaData(Map<String, String> metaMap) {

Review Comment:
   UT



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/rewrite/HoodieFileMetadataMerger.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.rewrite;

Review Comment:
   lets keep this one level higher.. `hudi.io.rewrite` ? naming wise : lets 
stick to one concept across the PR. 
   
   fastcopy or rewrite. and rename this `..hudi.io.fastcopy` ? 



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/parquet/io/HoodieParquetFileRewriter.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.parquet.io;
+
+import org.apache.hudi.io.storage.rewrite.HoodieFileMetadataMerger;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriter;
+import org.apache.hudi.storage.StoragePath;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.CompressionConverter;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+public class HoodieParquetFileRewriter extends HoodieParquetRewriterBase 
implements HoodieFileRewriter {

Review Comment:
   is this adapted from parquet-mr/parquet-java?
   



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/parquet/io/TestFile.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.parquet.io;
+
+import org.apache.parquet.example.data.simple.SimpleGroup;
+
+public class TestFile {

Review Comment:
   same.. please consolidate this clas and TestFileBuilder into sth 
maintainable/minimize.
   



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/rewrite/HoodieFileRewriterFactory.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.storage.StoragePath;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
+
+public class HoodieFileRewriterFactory {
+
+  private static HoodieFileRewriterFactory getWriterFactory(HoodieRecordType 
recordType, String extension) {
+    if (PARQUET.getFileExtension().equals(extension)) {
+      try {
+        Class<?> clazz = 
ReflectionUtils.getClass("org.apache.hudi.parquet.io.HoodieParquetRewriterFactory");
+        return (HoodieFileRewriterFactory) clazz.newInstance();
+      } catch (IllegalAccessException | IllegalArgumentException | 
InstantiationException e) {
+        throw new HoodieException("Unable to create hoodie avro parquet file 
writer factory", e);
+      }
+    }
+    throw new UnsupportedOperationException(extension + " file format not 
supported yet.");
+  }
+
+  public static <T, I, K, O> HoodieFileRewriter getFileRewriter(
+      List<StoragePath> inputFilePaths,
+      StoragePath targetFilePath,
+      Configuration conf,
+      HoodieConfig config,
+      HoodieFileMetadataMerger metadataMerger,
+      HoodieRecordType recordType,
+      Schema writeSchemaWithMetaFields) throws IOException {
+    String extension = FSUtils.getFileExtension(targetFilePath.getName());
+    HoodieFileRewriterFactory factory = getWriterFactory(recordType, 
extension);
+    return factory.newFileRewriter(inputFilePaths, targetFilePath, conf, 
config, metadataMerger, writeSchemaWithMetaFields);
+  }
+
+  protected <T> HoodieFileRewriter newFileRewriter(
+      List<StoragePath> inputFilePaths,
+      StoragePath targetFilePath,
+      Configuration conf,
+      HoodieConfig config,
+      HoodieFileMetadataMerger metadataMerger,
+      Schema writeSchemaWithMetaFields) throws IOException {
+    throw new UnsupportedOperationException();

Review Comment:
   msg?
   



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/parquet/io/TestFileBuilder.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.parquet.io;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.crypto.ParquetCipher;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+
+public class TestFileBuilder {

Review Comment:
   This seems like a test util and not an actual test? if this is borrowed from 
parquet.. can we move this along with other test utils.. and see if we can 
avoid copying over this much code. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateRewriteHandle.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.rewrite.HoodieFileMetadataMerger;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriter;
+import org.apache.hudi.io.storage.rewrite.HoodieFileRewriterFactory;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+public class HoodieCreateRewriteHandle<T, I, K, O> extends 
HoodieCreateHandle<T, I, K, O> {

Review Comment:
   +1. lets extend from HoodieWriteHandle and avoid the `initWriter` flags. 
   
   Naming: Prefer to call this sth like `HoodieFastCopyHandle` given all this 
is doing is creating a file out of input files. The CreateHandle takes input 
itr and create a file. So don't want to overload that naming here.
   
   idk if Deep vs Shallow copy is clear in this context, since shallow copy is 
not an option. we need to copy the actual bytes/records



##########
hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java:
##########
@@ -39,7 +39,7 @@ public TypedProperties() {
     super(null);
   }
 
-  protected TypedProperties(Properties defaults) {
+  public TypedProperties(Properties defaults) {

Review Comment:
   +1



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkStreamCopyClusteringExecutionStrategy.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.client.clustering.run.strategy;
+
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.ClusteringGroupInfo;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.io.HoodieCreateRewriteHandle;
+import org.apache.hudi.io.SingleFileHandleRewriteFactory;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+import org.apache.avro.Schema;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE;
+
+/**
+ * Clustering strategy to submit single spark jobs using streaming copy
+ * PAY ATTENTION!!!
+ * IN THIS STRATEGY
+ *  1. Only support clustering for cow table.
+ *  2. Sort function is not supported yet.
+ *  3. Each clustering group only has one task to write.
+ */
+public class SparkStreamCopyClusteringExecutionStrategy<T> extends 
SparkSortAndSizeExecutionStrategy<T> {

Review Comment:
   Good point



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkStreamCopyClusteringExecutionStrategy.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.client.clustering.run.strategy;
+
+import org.apache.hudi.avro.model.HoodieClusteringPlan;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.ClusteringGroupInfo;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.io.HoodieCreateRewriteHandle;
+import org.apache.hudi.io.SingleFileHandleRewriteFactory;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+import org.apache.avro.Schema;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+
+import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE;
+
+/**
+ * Clustering strategy to submit single spark jobs using streaming copy
+ * PAY ATTENTION!!!
+ * IN THIS STRATEGY
+ *  1. Only support clustering for cow table.
+ *  2. Sort function is not supported yet.

Review Comment:
   +1 on a new strategy. if sorting is not supported, this should not extend 
the SortAndSize..Strategy.. Let's cleanup code in a separate PR if you prefer 
to get this landed in a good place. 
   
   
   



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/rewrite/HoodieFileRewriter.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.rewrite;
+
+import java.io.IOException;
+
+public interface HoodieFileRewriter {

Review Comment:
   The interface shoud at-least cover `rewrite(inputFiles, outputFiles, cfg)` 
type methods?
   



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/rewrite/HoodieFileRewriter.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.rewrite;
+
+import java.io.IOException;
+
+public interface HoodieFileRewriter {

Review Comment:
   Lets not overload the `FileWriter` with this feature, since its not clear if 
we can handle this for  all file types? FileWriter works at the record iterator 
level . IMO deepCopy/fastCopy like implementations work one level lower than 
record level iterators writers do.. 
   
   I question if we need an interface esp in its current form where its just 
two methods that don't take any args. What other implementations do we want to 
have for the `HoodieFileRewriter`



-- 
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