suryaprasanna commented on code in PR #9006: URL: https://github.com/apache/hudi/pull/9006#discussion_r1244825343
########## 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; Review Comment: Fixed indentation. ########## 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"))); Review Comment: Moved to static final variables. ########## 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> { Review Comment: We have column prune and other encryption related implementation classes that use parquet-tools. Will need to check with other teams before pushing them to OSS. ########## 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: Reworded to include HoodieFileWriteHandle ########## 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: Done. ########## 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: We can improve this to include record_keys later, for now we can keep this simple to convert the parquet meta to write status. ########## 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: Added it as part of TODO statement. ########## 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: This PR is dependent on Marker changes PR from Balajee. So, for now added these comments will revert as soon as those changes are landed. ########## 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) { Review Comment: I think using srcPath is confusing changed it to oldFilePath, to keep it consistent with other handles. ########## 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: Done. ########## 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: Done. ########## 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: Since, the class runs on parquet_tools commands, so I thought ParquetToolsExecutionStrategy name might be better. By renaming it to EfficientParquetReWriteExecutionStrategy, we are reducing the emphasis on the ParquetTools. Let me know, what you think. ########## 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: parquet tools operate at file level. So, HoodieClusteringGroups that are created during clustering plan creation will take creating one group per file group. That way SingleSparkJobExecutionStrategy class can parallelize the execution on all these file groups in parallel. Later when including other tools like merge etc we can relax this condition. ########## 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: Added it. ########## 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: Yeah, corrected it. ########## 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: Since the class is ParquetToolsExecutionStrategy I thought executeTools might be better here since it will actually run parqeut_tools commands here. Let me know what you think. -- 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]
