nsivabalan commented on code in PR #9006:
URL: https://github.com/apache/hudi/pull/9006#discussion_r1234348561


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/ParquetFileMetaToWriteStatusConvertor.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.execution;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This class is mainly used by the ParquetToolsExecutionStrategy to generate 
WriteStatus classes.
+ */
+public class ParquetFileMetaToWriteStatusConvertor<T extends 
HoodieRecordPayload, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ParquetFileMetaToWriteStatusConvertor.class);
+  private final HoodieTable<T,I,K,O> hoodieTable;
+  private final HoodieWriteConfig writeConfig;
+  private final FileSystem fs;
+
+  public ParquetFileMetaToWriteStatusConvertor(HoodieTable<T, I, K, O> 
hoodieTable, HoodieWriteConfig writeConfig) {

Review Comment:
   awesome. we might need something like this to support insert_overwrite, 
delete_partition w/ RLI. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieFileWriteHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.execution.ParquetFileMetaToWriteStatusConvertor;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Write handle that is used to work on top of files rather than on individual 
records.
+ */
+public class HoodieFileWriteHandler<T extends HoodieRecordPayload, I, K, O> 
extends HoodieWriteHandle<T, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieFileWriteHandler.class);
+  private final Path path;
+  private String prevCommit;
+
+  public HoodieFileWriteHandler(HoodieWriteConfig config, String instantTime, 
HoodieTable<T, I, K, O> hoodieTable,
+                                String partitionPath, String fileId, 
TaskContextSupplier taskContextSupplier,
+                                Path srcPath) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, 
taskContextSupplier);
+
+    // Output file path.
+    this.path = makeNewPath(partitionPath);
+    this.prevCommit = srcPath.getName().split("_")[2].split("\\.")[0];
+
+    // Create inProgress marker file
+    createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, 
this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    // TODO: Create inprogress marker here and remove above marker file 
creation, once the marker PR is landed.
+    // 
createInProgressMarkerFile(partitionPath,FSUtils.makeDataFileName(this.instantTime,
 this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    LOG.info("New CreateHandle for partition :" + partitionPath + " with 
fileId " + fileId);
+  }
+
+  /**
+   * Complete writing of the file by creating the success marker file.
+   * @return WriteStatuses, ideally it will be only one object.
+   */
+  @Override
+  public List<WriteStatus> close() {
+    LOG.info("Closing the file " + this.fileId + " as we are done with the 
file.");
+    try {
+      Map<String, Object> executionConfigs = new HashMap<>();
+      executionConfigs.put("prevCommit", prevCommit);
+      executionConfigs.put("timeTaken", timer.endTimer());
+
+      this.writeStatus = generateWriteStatus(path.toString(), partitionPath, 
executionConfigs);
+
+      // TODO: Create completed marker file here once the marker PR is landed.
+      // createCompleteMarkerFile throws hoodieException, if marker directory 
is not present.
+      // createCompletedMarkerFile(partitionPath);
+      LOG.info(String.format("CreateHandle for partitionPath %s fileID %s, 
took %d ms.",

Review Comment:
   again, fix logging 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieFileWriteHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.execution.ParquetFileMetaToWriteStatusConvertor;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Write handle that is used to work on top of files rather than on individual 
records.
+ */
+public class HoodieFileWriteHandler<T extends HoodieRecordPayload, I, K, O> 
extends HoodieWriteHandle<T, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieFileWriteHandler.class);
+  private final Path path;
+  private String prevCommit;
+
+  public HoodieFileWriteHandler(HoodieWriteConfig config, String instantTime, 
HoodieTable<T, I, K, O> hoodieTable,
+                                String partitionPath, String fileId, 
TaskContextSupplier taskContextSupplier,
+                                Path srcPath) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, 
taskContextSupplier);
+
+    // Output file path.
+    this.path = makeNewPath(partitionPath);
+    this.prevCommit = srcPath.getName().split("_")[2].split("\\.")[0];
+
+    // Create inProgress marker file
+    createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, 
this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    // TODO: Create inprogress marker here and remove above marker file 
creation, once the marker PR is landed.
+    // 
createInProgressMarkerFile(partitionPath,FSUtils.makeDataFileName(this.instantTime,
 this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    LOG.info("New CreateHandle for partition :" + partitionPath + " with 
fileId " + fileId);
+  }
+
+  /**
+   * Complete writing of the file by creating the success marker file.
+   * @return WriteStatuses, ideally it will be only one object.
+   */
+  @Override
+  public List<WriteStatus> close() {
+    LOG.info("Closing the file " + this.fileId + " as we are done with the 
file.");
+    try {
+      Map<String, Object> executionConfigs = new HashMap<>();
+      executionConfigs.put("prevCommit", prevCommit);
+      executionConfigs.put("timeTaken", timer.endTimer());
+
+      this.writeStatus = generateWriteStatus(path.toString(), partitionPath, 
executionConfigs);
+
+      // TODO: Create completed marker file here once the marker PR is landed.
+      // createCompleteMarkerFile throws hoodieException, if marker directory 
is not present.

Review Comment:
   please do add jira id here 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieFileWriteHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.execution.ParquetFileMetaToWriteStatusConvertor;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Write handle that is used to work on top of files rather than on individual 
records.
+ */
+public class HoodieFileWriteHandler<T extends HoodieRecordPayload, I, K, O> 
extends HoodieWriteHandle<T, I, K, O> {

Review Comment:
   rename to "HoodieFileWriteHandle". no "r" in the end. all of your handles 
are named this way. lets not add "r" in the end. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/ParquetFileMetaToWriteStatusConvertor.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.execution;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This class is mainly used by the ParquetToolsExecutionStrategy to generate 
WriteStatus classes.
+ */
+public class ParquetFileMetaToWriteStatusConvertor<T extends 
HoodieRecordPayload, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ParquetFileMetaToWriteStatusConvertor.class);
+  private final HoodieTable<T,I,K,O> hoodieTable;
+  private final HoodieWriteConfig writeConfig;
+  private final FileSystem fs;
+
+  public ParquetFileMetaToWriteStatusConvertor(HoodieTable<T, I, K, O> 
hoodieTable, HoodieWriteConfig writeConfig) {
+    this.hoodieTable = hoodieTable;
+    this.writeConfig = writeConfig;
+    this.fs = this.hoodieTable.getMetaClient().getFs();
+  }
+
+  /**
+   * This method generates writeStatus object from parquet file.
+   */
+  public WriteStatus convert(String parquetFile, String partitionPath,
+                             Map<String, Object> executionConfigs) throws 
IOException {
+    LOG.info("Creating write status for parquet file " + parquetFile);
+    WriteStatus writeStatus = (WriteStatus) 
ReflectionUtils.loadClass(this.writeConfig.getWriteStatusClassName(),
+        !this.hoodieTable.getIndex().isImplicitWithStorage(), 
this.writeConfig.getWriteStatusFailureFraction());
+    Path parquetFilePath = new Path(parquetFile);
+    writeStatus.setFileId(FSUtils.getFileId(parquetFilePath.getName()));
+    writeStatus.setPartitionPath(partitionPath);
+    generateHoodieWriteStat(writeStatus, parquetFilePath, executionConfigs);
+    return writeStatus;
+  }
+
+  /**
+   * This method generates HoodieWriteStat object and set it as part of 
WriteStatus object.
+   */
+  private void generateHoodieWriteStat(
+      WriteStatus writeStatus, Path parquetFilePath, Map<String, Object> 
executionConfigs) throws IOException {
+    HoodieWriteStat stat = new HoodieWriteStat();
+
+    // Set row count
+    ParquetMetadata parquetMetadata = 
ParquetFileReader.readFooter(this.fs.getConf(), parquetFilePath,
+        ParquetMetadataConverter.NO_FILTER);
+    List<BlockMetaData> blockMetaDataList = parquetMetadata.getBlocks();
+    long rowCount = 
blockMetaDataList.stream().mapToLong(BlockMetaData::getRowCount).sum();
+    stat.setNumWrites(rowCount);
+    stat.setNumInserts(rowCount);
+
+    // Set runtime stats
+    HoodieWriteStat.RuntimeStats runtimeStats = new 
HoodieWriteStat.RuntimeStats();
+    runtimeStats.setTotalCreateTime((long) executionConfigs.get("timeTaken"));
+    stat.setRuntimeStats(runtimeStats);
+
+    // File size
+    FileStatus parquetFileStatus = this.fs.getFileStatus(parquetFilePath);
+    long fileSize = parquetFileStatus.getLen();
+    stat.setFileSizeInBytes(fileSize);
+    stat.setTotalWriteBytes(fileSize);
+
+    stat.setFileId(writeStatus.getFileId());
+    stat.setPartitionPath(writeStatus.getPartitionPath());
+    stat.setPath(new Path(writeConfig.getBasePath()), parquetFilePath);
+    stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
+    stat.setPrevCommit(String.valueOf(executionConfigs.get("prevCommit")));
+
+    writeStatus.setStat(stat);
+  }

Review Comment:
   oh this is not populating any succ records. anyways, we can build something 
on top of this for insert_overwrite support



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/ParquetToolsExecutionStrategy.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.avro.Schema;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.ClusteringGroupInfo;
+import org.apache.hudi.common.model.ClusteringOperation;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieClusteringException;
+import org.apache.hudi.io.HoodieFileWriteHandler;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+/**
+ * This class gives skeleton implementation for set of clustering execution 
strategy
+ * that use parquet-tools commands.
+ */
+public abstract class ParquetToolsExecutionStrategy<T extends 
HoodieRecordPayload<T>>
+    extends SingleSparkJobExecutionStrategy<T> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ParquetToolsExecutionStrategy.class);
+
+  public ParquetToolsExecutionStrategy(
+      HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig 
writeConfig) {
+    super(table, engineContext, writeConfig);
+  }
+
+  protected Stream<WriteStatus> runClusteringForGroup(ClusteringGroupInfo 
clusteringOps, Map<String, String> strategyParams,
+                                                      boolean 
preserveHoodieMetadata, SerializableSchema schema,
+                                                      TaskContextSupplier 
taskContextSupplier, String instantTime) {
+    LOG.info("Starting clustering operation on input file ids.");
+    List<ClusteringOperation> clusteringOperations = 
clusteringOps.getOperations();
+    if (clusteringOperations.size() > 1) {
+      throw new HoodieClusteringException("Expect only one clustering 
operation during rewrite: " + getClass().getName());
+    }
+
+    ClusteringOperation clusteringOperation = clusteringOperations.get(0);
+    String fileId = clusteringOperation.getFileId();
+    String partitionPath = clusteringOperation.getPartitionPath();
+    String dataFilePath = clusteringOperation.getDataFilePath();
+    Path srcPath = new Path(dataFilePath);
+    HoodieFileWriteHandler writeHandler = new 
HoodieFileWriteHandler(getWriteConfig(), instantTime, getHoodieTable(),
+        partitionPath, fileId, taskContextSupplier, srcPath);
+
+    // Executes the parquet-tools command.
+    executeTools(srcPath, writeHandler.getPath());
+    return writeHandler.close().stream();
+  }
+
+  /**
+   * This method needs to be overridden by the child classes.
+   * In this method parquet-tools command can be created and executed.
+   * Assuming that the parquet-tools command operate per file basis this 
interface allows command to run once per file.
+   */
+  protected abstract void executeTools(Path srcFilePath, Path destFilePath);

Review Comment:
   executeRewrite or executeConvert 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieFileWriteHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.execution.ParquetFileMetaToWriteStatusConvertor;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Write handle that is used to work on top of files rather than on individual 
records.
+ */
+public class HoodieFileWriteHandler<T extends HoodieRecordPayload, I, K, O> 
extends HoodieWriteHandle<T, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieFileWriteHandler.class);
+  private final Path path;
+  private String prevCommit;
+
+  public HoodieFileWriteHandler(HoodieWriteConfig config, String instantTime, 
HoodieTable<T, I, K, O> hoodieTable,
+                                String partitionPath, String fileId, 
TaskContextSupplier taskContextSupplier,
+                                Path srcPath) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, 
taskContextSupplier);
+
+    // Output file path.
+    this.path = makeNewPath(partitionPath);
+    this.prevCommit = srcPath.getName().split("_")[2].split("\\.")[0];
+
+    // Create inProgress marker file
+    createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, 
this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    // TODO: Create inprogress marker here and remove above marker file 
creation, once the marker PR is landed.
+    // 
createInProgressMarkerFile(partitionPath,FSUtils.makeDataFileName(this.instantTime,
 this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    LOG.info("New CreateHandle for partition :" + partitionPath + " with 
fileId " + fileId);

Review Comment:
   fix logging. this is no CreateHandle



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieFileWriteHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.execution.ParquetFileMetaToWriteStatusConvertor;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Write handle that is used to work on top of files rather than on individual 
records.
+ */
+public class HoodieFileWriteHandler<T extends HoodieRecordPayload, I, K, O> 
extends HoodieWriteHandle<T, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieFileWriteHandler.class);
+  private final Path path;
+  private String prevCommit;
+
+  public HoodieFileWriteHandler(HoodieWriteConfig config, String instantTime, 
HoodieTable<T, I, K, O> hoodieTable,
+                                String partitionPath, String fileId, 
TaskContextSupplier taskContextSupplier,
+                                Path srcPath) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, 
taskContextSupplier);
+
+    // Output file path.
+    this.path = makeNewPath(partitionPath);
+    this.prevCommit = srcPath.getName().split("_")[2].split("\\.")[0];
+
+    // Create inProgress marker file
+    createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, 
this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    // TODO: Create inprogress marker here and remove above marker file 
creation, once the marker PR is landed.
+    // 
createInProgressMarkerFile(partitionPath,FSUtils.makeDataFileName(this.instantTime,
 this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    LOG.info("New CreateHandle for partition :" + partitionPath + " with 
fileId " + fileId);
+  }
+
+  /**
+   * Complete writing of the file by creating the success marker file.
+   * @return WriteStatuses, ideally it will be only one object.
+   */
+  @Override
+  public List<WriteStatus> close() {
+    LOG.info("Closing the file " + this.fileId + " as we are done with the 
file.");
+    try {
+      Map<String, Object> executionConfigs = new HashMap<>();
+      executionConfigs.put("prevCommit", prevCommit);
+      executionConfigs.put("timeTaken", timer.endTimer());
+
+      this.writeStatus = generateWriteStatus(path.toString(), partitionPath, 
executionConfigs);
+
+      // TODO: Create completed marker file here once the marker PR is landed.
+      // createCompleteMarkerFile throws hoodieException, if marker directory 
is not present.
+      // createCompletedMarkerFile(partitionPath);
+      LOG.info(String.format("CreateHandle for partitionPath %s fileID %s, 
took %d ms.",
+          writeStatus.getStat().getPartitionPath(), 
writeStatus.getStat().getFileId(),
+          writeStatus.getStat().getRuntimeStats().getTotalCreateTime()));
+
+      return Collections.singletonList(writeStatus);
+    } catch (IOException e) {
+      throw new HoodieInsertException("Failed to close the Insert Handle for 
path " + path, e);

Review Comment:
   Insert handle ?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieFileWriteHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.execution.ParquetFileMetaToWriteStatusConvertor;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Write handle that is used to work on top of files rather than on individual 
records.
+ */
+public class HoodieFileWriteHandler<T extends HoodieRecordPayload, I, K, O> 
extends HoodieWriteHandle<T, I, K, O> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieFileWriteHandler.class);
+  private final Path path;
+  private String prevCommit;
+
+  public HoodieFileWriteHandler(HoodieWriteConfig config, String instantTime, 
HoodieTable<T, I, K, O> hoodieTable,
+                                String partitionPath, String fileId, 
TaskContextSupplier taskContextSupplier,
+                                Path srcPath) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, 
taskContextSupplier);
+
+    // Output file path.
+    this.path = makeNewPath(partitionPath);
+    this.prevCommit = srcPath.getName().split("_")[2].split("\\.")[0];
+
+    // Create inProgress marker file
+    createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, 
this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
+    // TODO: Create inprogress marker here and remove above marker file 
creation, once the marker PR is landed.

Review Comment:
   can you add the jira number here please



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieFileWriteHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.execution.ParquetFileMetaToWriteStatusConvertor;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Write handle that is used to work on top of files rather than on individual 
records.
+ */
+public class HoodieFileWriteHandler<T extends HoodieRecordPayload, I, K, O> 
extends HoodieWriteHandle<T, I, K, O> {
+

Review Comment:
   I am also thinking, if we can name this 
   HoodieFileRewriteHandle 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/ParquetToolsExecutionStrategy.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.avro.Schema;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.ClusteringGroupInfo;
+import org.apache.hudi.common.model.ClusteringOperation;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieClusteringException;
+import org.apache.hudi.io.HoodieFileWriteHandler;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+/**
+ * This class gives skeleton implementation for set of clustering execution 
strategy
+ * that use parquet-tools commands.
+ */
+public abstract class ParquetToolsExecutionStrategy<T extends 
HoodieRecordPayload<T>>

Review Comment:
   should we name this class as EfficientParquetReWriteExecutionStrategy 
   
   embedding Parquettools in the class name somehow does not sit well.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/ParquetToolsExecutionStrategy.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.avro.Schema;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.ClusteringGroupInfo;
+import org.apache.hudi.common.model.ClusteringOperation;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieClusteringException;
+import org.apache.hudi.io.HoodieFileWriteHandler;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+/**
+ * This class gives skeleton implementation for set of clustering execution 
strategy
+ * that use parquet-tools commands.
+ */
+public abstract class ParquetToolsExecutionStrategy<T extends 
HoodieRecordPayload<T>>
+    extends SingleSparkJobExecutionStrategy<T> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ParquetToolsExecutionStrategy.class);
+
+  public ParquetToolsExecutionStrategy(
+      HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig 
writeConfig) {
+    super(table, engineContext, writeConfig);
+  }
+
+  protected Stream<WriteStatus> runClusteringForGroup(ClusteringGroupInfo 
clusteringOps, Map<String, String> strategyParams,
+                                                      boolean 
preserveHoodieMetadata, SerializableSchema schema,
+                                                      TaskContextSupplier 
taskContextSupplier, String instantTime) {
+    LOG.info("Starting clustering operation on input file ids.");
+    List<ClusteringOperation> clusteringOperations = 
clusteringOps.getOperations();
+    if (clusteringOperations.size() > 1) {
+      throw new HoodieClusteringException("Expect only one clustering 
operation during rewrite: " + getClass().getName());

Review Comment:
   whey throw if we have more than 1 CO? 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/ParquetToolsExecutionStrategy.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.avro.Schema;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.model.ClusteringGroupInfo;
+import org.apache.hudi.common.model.ClusteringOperation;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieClusteringException;
+import org.apache.hudi.io.HoodieFileWriteHandler;
+import org.apache.hudi.table.HoodieTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+/**
+ * This class gives skeleton implementation for set of clustering execution 
strategy
+ * that use parquet-tools commands.
+ */
+public abstract class ParquetToolsExecutionStrategy<T extends 
HoodieRecordPayload<T>>
+    extends SingleSparkJobExecutionStrategy<T> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ParquetToolsExecutionStrategy.class);
+
+  public ParquetToolsExecutionStrategy(
+      HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig 
writeConfig) {
+    super(table, engineContext, writeConfig);
+  }
+
+  protected Stream<WriteStatus> runClusteringForGroup(ClusteringGroupInfo 
clusteringOps, Map<String, String> strategyParams,
+                                                      boolean 
preserveHoodieMetadata, SerializableSchema schema,
+                                                      TaskContextSupplier 
taskContextSupplier, String instantTime) {
+    LOG.info("Starting clustering operation on input file ids.");
+    List<ClusteringOperation> clusteringOperations = 
clusteringOps.getOperations();
+    if (clusteringOperations.size() > 1) {
+      throw new HoodieClusteringException("Expect only one clustering 
operation during rewrite: " + getClass().getName());
+    }
+
+    ClusteringOperation clusteringOperation = clusteringOperations.get(0);
+    String fileId = clusteringOperation.getFileId();
+    String partitionPath = clusteringOperation.getPartitionPath();
+    String dataFilePath = clusteringOperation.getDataFilePath();
+    Path srcPath = new Path(dataFilePath);
+    HoodieFileWriteHandler writeHandler = new 
HoodieFileWriteHandler(getWriteConfig(), instantTime, getHoodieTable(),
+        partitionPath, fileId, taskContextSupplier, srcPath);
+
+    // Executes the parquet-tools command.
+    executeTools(srcPath, writeHandler.getPath());
+    return writeHandler.close().stream();
+  }
+
+  /**
+   * This method needs to be overridden by the child classes.
+   * In this method parquet-tools command can be created and executed.
+   * Assuming that the parquet-tools command operate per file basis this 
interface allows command to run once per file.
+   */
+  protected abstract void executeTools(Path srcFilePath, Path destFilePath);
+
+  /**
+   * Since parquet-tools works at the file level, this method need not be used 
overridden.
+   */
+  @Override
+  public Iterator<List<WriteStatus>> performClusteringWithRecordsIterator(
+      final Iterator<HoodieRecord<T>> records, final int numOutputGroups, 
final String instantTime,
+      final Map<String, String> strategyParams, final Schema schema, final 
List<HoodieFileGroupId> fileGroupIdList,
+      final boolean preserveHoodieMetadata, final TaskContextSupplier 
taskContextSupplier) {
+    return null;

Review Comment:
   then, should we throw here then. 



-- 
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: commits-unsubscr...@hudi.apache.org

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


Reply via email to