phet commented on code in PR #4058:
URL: https://github.com/apache/gobblin/pull/4058#discussion_r1793834712


##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergOverwritePartitionsStep.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.github.rholder.retry.Attempt;
+import com.github.rholder.retry.RetryException;
+import com.github.rholder.retry.RetryListener;
+import com.github.rholder.retry.Retryer;
+import com.google.common.collect.ImmutableMap;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.commit.CommitStep;
+import org.apache.gobblin.util.retry.RetryerFactory;
+
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_INTERVAL_MS;
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_TIMES;
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_TYPE;
+import static org.apache.gobblin.util.retry.RetryerFactory.RetryType;
+
+/**
+ * Commit step for overwriting partitions in an Iceberg table.
+ * <p>
+ * This class implements the {@link CommitStep} interface and provides 
functionality to overwrite
+ * partitions in the destination Iceberg table using serialized data files.
+ * </p>
+ */
+@Slf4j
+public class IcebergOverwritePartitionsStep implements CommitStep {
+  private final String destTableIdStr;
+  private final Properties properties;
+  private final byte[] serializedDataFiles;
+  private final String partitionColName;
+  private final String partitionValue;
+  public static final String OVERWRITE_PARTITIONS_RETRYER_CONFIG_PREFIX = 
IcebergDatasetFinder.ICEBERG_DATASET_PREFIX +
+      ".catalog.overwrite.partitions.retries";
+  private static final Config RETRYER_FALLBACK_CONFIG = 
ConfigFactory.parseMap(ImmutableMap.of(
+      RETRY_INTERVAL_MS, TimeUnit.SECONDS.toMillis(3L),
+      RETRY_TIMES, 3,
+      RETRY_TYPE, RetryType.FIXED_ATTEMPT.name()));
+
+  /**
+   * Constructs an {@code IcebergReplacePartitionsStep} with the specified 
parameters.
+   *
+   * @param destTableIdStr the identifier of the destination table as a string
+   * @param serializedDataFiles [from List<DataFiles>] the serialized data 
files to be used for replacing partitions
+   * @param properties the properties containing configuration
+   */
+  public IcebergOverwritePartitionsStep(String destTableIdStr, String 
partitionColName, String partitionValue, byte[] serializedDataFiles, Properties 
properties) {
+    this.destTableIdStr = destTableIdStr;
+    this.partitionColName = partitionColName;
+    this.partitionValue = partitionValue;
+    this.serializedDataFiles = serializedDataFiles;
+    this.properties = properties;
+  }
+
+  @Override
+  public boolean isCompleted() {
+    return false;
+  }
+
+  /**
+   * Executes the partition replacement in the destination Iceberg table.
+   * Also, have retry mechanism as done in {@link 
IcebergRegisterStep#execute()}
+   *
+   * @throws IOException if an I/O error occurs during execution
+   */
+  @Override
+  public void execute() throws IOException {
+    IcebergTable destTable = 
createDestinationCatalog().openTable(TableIdentifier.parse(this.destTableIdStr));
+    List<DataFile> dataFiles = 
SerializationUtil.deserializeFromBytes(this.serializedDataFiles);
+    try {
+      log.info("Overwriting Data files of partition {} with value {} for 
destination table : {} ",
+          this.partitionColName,
+          this.partitionValue,
+          this.destTableIdStr
+      );
+      Retryer<Void> overwritePartitionsRetryer = 
createOverwritePartitionsRetryer();
+      overwritePartitionsRetryer.call(() -> {
+        destTable.overwritePartition(dataFiles, this.partitionColName, 
this.partitionValue);
+        return null;
+      });
+      log.info("Overwriting Data files completed for partition {} with value 
{} for destination table : {} ",

Review Comment:
   I almost thought this was a duplicated comment, probably because it's so 
wordy/conversational.  let's streamline.
   
   also, `IcebergRegisterStep` and `IcebergDataset` all follow the convention 
of naming the table at the start of each log msg (in a `~{}~ ` prefix).
   
   how about:
   ```
   "~{}~ Attempting partition overwrite - partition: {}; value: {}; 
numDataFiles: {}; path[0]: {}"
   // or "Starting" or "Preparing"
   ...
   "~{}~ Successful partition overwrite - partition: {}; value: {}"
   ```



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergPartitionDataset.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.ImmutableList;
+
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.data.management.copy.CopyConfiguration;
+import org.apache.gobblin.data.management.copy.CopyEntity;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
+import org.apache.gobblin.data.management.copy.CopyableDataset;
+import org.apache.gobblin.util.measurement.GrowthMilestoneTracker;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergMatchesAnyPropNamePartitionFilterPredicate;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergPartitionFilterPredicateUtil;
+
+/**
+ * Iceberg Partition dataset implementing {@link CopyableDataset}
+ * <p>
+ * This class extends {@link IcebergDataset} and provides functionality to 
filter partitions
+ * and generate copy entities for partition based data movement.
+ * </p>
+ */
+@Slf4j
+public class IcebergPartitionDataset extends IcebergDataset {
+  private static final List<String> supportedTransforms = 
ImmutableList.of("identity", "truncate");
+  private final Predicate<StructLike> partitionFilterPredicate;
+  private final Map<Path, Path> srcPathToDestPath;
+  private final String partitionColumnName;
+  private final String partitionColValue;
+
+  public IcebergPartitionDataset(IcebergTable srcIcebergTable, IcebergTable 
destIcebergTable, Properties properties,
+      FileSystem sourceFs, boolean shouldIncludeMetadataPath, String 
partitionColumnName, String partitionColValue)
+      throws IcebergTable.TableNotFoundException {
+    super(srcIcebergTable, destIcebergTable, properties, sourceFs, 
shouldIncludeMetadataPath);
+    this.partitionColumnName = partitionColumnName;
+    this.partitionColValue = partitionColValue;
+    this.partitionFilterPredicate = createPartitionFilterPredicate();
+    this.srcPathToDestPath = new HashMap<>();
+  }
+
+  private Predicate<StructLike> createPartitionFilterPredicate() throws 
IcebergTable.TableNotFoundException {
+    //TODO: Refactor it later using factory or other way to support different 
types of filter predicate
+    // Also take into consideration creation of Expression Filter to be used 
in overwrite api
+    TableMetadata srcTableMetadata = 
getSrcIcebergTable().accessTableMetadata();
+    int partitionColumnIndex = 
IcebergPartitionFilterPredicateUtil.getPartitionColumnIndex(
+        this.partitionColumnName,
+        srcTableMetadata,
+        supportedTransforms
+    );
+    Preconditions.checkArgument(partitionColumnIndex >= 0, String.format(
+        "Partition column %s not found in table %s",
+        this.partitionColumnName, this.getFileSetId()));
+    return new 
IcebergMatchesAnyPropNamePartitionFilterPredicate(partitionColumnIndex, 
this.partitionColValue);
+  }

Review Comment:
   these two seem closely related enough that `getPartitionColumnIndex` could 
be a static in `IcebergMatchesAnyPropNamePartitionFilterPredicate`.  the call 
and the `Precondition` could even be encapsulated by a factory method:
   ```
   class IcebergMatchesAnyP {
       public static IcebergMatchesAnyP forColumnName(
           String columnName,
           TableMetadata md,
           List<String> supportedTransforms, 
           String columnValue) {
         ...
     }
   ```



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergOverwritePartitionsStep.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.github.rholder.retry.Attempt;
+import com.github.rholder.retry.RetryException;
+import com.github.rholder.retry.RetryListener;
+import com.github.rholder.retry.Retryer;
+import com.google.common.collect.ImmutableMap;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.commit.CommitStep;
+import org.apache.gobblin.util.retry.RetryerFactory;
+
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_INTERVAL_MS;
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_TIMES;
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_TYPE;
+import static org.apache.gobblin.util.retry.RetryerFactory.RetryType;
+
+/**
+ * Commit step for overwriting partitions in an Iceberg table.
+ * <p>
+ * This class implements the {@link CommitStep} interface and provides 
functionality to overwrite
+ * partitions in the destination Iceberg table using serialized data files.
+ * </p>
+ */
+@Slf4j
+public class IcebergOverwritePartitionsStep implements CommitStep {
+  private final String destTableIdStr;
+  private final Properties properties;
+  private final byte[] serializedDataFiles;
+  private final String partitionColName;
+  private final String partitionValue;
+  public static final String OVERWRITE_PARTITIONS_RETRYER_CONFIG_PREFIX = 
IcebergDatasetFinder.ICEBERG_DATASET_PREFIX +
+      ".catalog.overwrite.partitions.retries";
+  private static final Config RETRYER_FALLBACK_CONFIG = 
ConfigFactory.parseMap(ImmutableMap.of(
+      RETRY_INTERVAL_MS, TimeUnit.SECONDS.toMillis(3L),
+      RETRY_TIMES, 3,
+      RETRY_TYPE, RetryType.FIXED_ATTEMPT.name()));
+
+  /**
+   * Constructs an {@code IcebergReplacePartitionsStep} with the specified 
parameters.
+   *
+   * @param destTableIdStr the identifier of the destination table as a string
+   * @param serializedDataFiles [from List<DataFiles>] the serialized data 
files to be used for replacing partitions
+   * @param properties the properties containing configuration
+   */
+  public IcebergOverwritePartitionsStep(String destTableIdStr, String 
partitionColName, String partitionValue, byte[] serializedDataFiles, Properties 
properties) {
+    this.destTableIdStr = destTableIdStr;
+    this.partitionColName = partitionColName;
+    this.partitionValue = partitionValue;
+    this.serializedDataFiles = serializedDataFiles;
+    this.properties = properties;
+  }
+
+  @Override
+  public boolean isCompleted() {
+    return false;
+  }
+
+  /**
+   * Executes the partition replacement in the destination Iceberg table.
+   * Also, have retry mechanism as done in {@link 
IcebergRegisterStep#execute()}
+   *
+   * @throws IOException if an I/O error occurs during execution
+   */
+  @Override
+  public void execute() throws IOException {

Review Comment:
   I realize we would not have equal metadata between the source and dest 
tables, but do we have any concerns around concurrent modification of the dest, 
between the time `IcebergDataset::generateCopyEntities` runs and this 
`CommitStep` does?
   
   in `IcebergRegisterStep`, we detect and abort if the dest table has been 
modified in the meanwhile.  I can understand some cases where we might here 
allow concurrent writes, but do we want it in all cases?  (the policy could be 
controlled by a config.)
   
   if we actually believe unequivocal overwrite to be a benefit, let's leave a 
code comment to document that intent



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergTable.java:
##########
@@ -217,4 +227,59 @@ protected void registerIcebergTable(TableMetadata 
srcMetadata, TableMetadata dst
       this.tableOps.commit(dstMetadata, srcMetadata);
     }
   }
+
+  /**
+   * Retrieves a list of data files from the current snapshot that match the 
specified partition filter predicate.
+   *
+   * @param icebergPartitionFilterPredicate the predicate to filter partitions
+   * @return a list of data files that match the partition filter predicate
+   * @throws IOException if an I/O error occurs while accessing the table 
metadata or reading manifest files
+   */
+  public List<DataFile> getPartitionSpecificDataFiles(Predicate<StructLike> 
icebergPartitionFilterPredicate) throws IOException {
+    TableMetadata tableMetadata = accessTableMetadata();
+    Snapshot currentSnapshot = tableMetadata.currentSnapshot();
+    log.info("Starting to copy data files from snapshot: {}", 
currentSnapshot.snapshotId());
+    //TODO: Add support for deleteManifests as well later
+    // Currently supporting dataManifests only
+    List<ManifestFile> dataManifestFiles = 
currentSnapshot.dataManifests(this.tableOps.io());
+    List<DataFile> dataFileList = new ArrayList<>();
+    for (ManifestFile manifestFile : dataManifestFiles) {
+      try (ManifestReader<DataFile> manifestReader = 
ManifestFiles.read(manifestFile, this.tableOps.io());
+          CloseableIterator<DataFile> dataFiles = manifestReader.iterator()) {
+        dataFiles.forEachRemaining(dataFile -> {
+          if (icebergPartitionFilterPredicate.test(dataFile.partition())) {
+            dataFileList.add(dataFile.copy());
+          }
+        });
+      } catch (IOException e) {
+        log.warn("Failed to read manifest file: {} " , manifestFile.path(), e);
+      }

Review Comment:
   iceberg is atomic/transactional, so I really don't agree w/ swallowing 
exceptions and still proceeding onward when the table is corrupted.  that has 
the potential for us to lay even more corruption on top of that... 
   
   please explain if you see a genuine argument for ignoring errors.



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergPartitionDataset.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.ImmutableList;
+
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.data.management.copy.CopyConfiguration;
+import org.apache.gobblin.data.management.copy.CopyEntity;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
+import org.apache.gobblin.data.management.copy.CopyableDataset;
+import org.apache.gobblin.util.measurement.GrowthMilestoneTracker;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergMatchesAnyPropNamePartitionFilterPredicate;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergPartitionFilterPredicateUtil;
+
+/**
+ * Iceberg Partition dataset implementing {@link CopyableDataset}
+ * <p>
+ * This class extends {@link IcebergDataset} and provides functionality to 
filter partitions
+ * and generate copy entities for partition based data movement.
+ * </p>
+ */
+@Slf4j
+public class IcebergPartitionDataset extends IcebergDataset {
+  private static final List<String> supportedTransforms = 
ImmutableList.of("identity", "truncate");
+  private final Predicate<StructLike> partitionFilterPredicate;
+  private final Map<Path, Path> srcPathToDestPath;
+  private final String partitionColumnName;
+  private final String partitionColValue;
+
+  public IcebergPartitionDataset(IcebergTable srcIcebergTable, IcebergTable 
destIcebergTable, Properties properties,
+      FileSystem sourceFs, boolean shouldIncludeMetadataPath, String 
partitionColumnName, String partitionColValue)
+      throws IcebergTable.TableNotFoundException {
+    super(srcIcebergTable, destIcebergTable, properties, sourceFs, 
shouldIncludeMetadataPath);
+    this.partitionColumnName = partitionColumnName;
+    this.partitionColValue = partitionColValue;
+    this.partitionFilterPredicate = createPartitionFilterPredicate();
+    this.srcPathToDestPath = new HashMap<>();
+  }
+
+  private Predicate<StructLike> createPartitionFilterPredicate() throws 
IcebergTable.TableNotFoundException {

Review Comment:
   put `private` methods at the tail of the class



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergPartitionDataset.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.ImmutableList;
+
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.data.management.copy.CopyConfiguration;
+import org.apache.gobblin.data.management.copy.CopyEntity;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
+import org.apache.gobblin.data.management.copy.CopyableDataset;
+import org.apache.gobblin.util.measurement.GrowthMilestoneTracker;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergMatchesAnyPropNamePartitionFilterPredicate;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergPartitionFilterPredicateUtil;
+
+/**
+ * Iceberg Partition dataset implementing {@link CopyableDataset}
+ * <p>
+ * This class extends {@link IcebergDataset} and provides functionality to 
filter partitions
+ * and generate copy entities for partition based data movement.
+ * </p>
+ */
+@Slf4j
+public class IcebergPartitionDataset extends IcebergDataset {
+  private static final List<String> supportedTransforms = 
ImmutableList.of("identity", "truncate");

Review Comment:
   perhaps a comment on why these in particular and either a TODO about making 
this configurable or just explain why hard-coding makes sense.



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergOverwritePartitionsStep.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.github.rholder.retry.Attempt;
+import com.github.rholder.retry.RetryException;
+import com.github.rholder.retry.RetryListener;
+import com.github.rholder.retry.Retryer;
+import com.google.common.collect.ImmutableMap;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.commit.CommitStep;
+import org.apache.gobblin.util.retry.RetryerFactory;
+
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_INTERVAL_MS;
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_TIMES;
+import static org.apache.gobblin.util.retry.RetryerFactory.RETRY_TYPE;
+import static org.apache.gobblin.util.retry.RetryerFactory.RetryType;
+
+/**
+ * Commit step for overwriting partitions in an Iceberg table.
+ * <p>
+ * This class implements the {@link CommitStep} interface and provides 
functionality to overwrite
+ * partitions in the destination Iceberg table using serialized data files.
+ * </p>
+ */
+@Slf4j
+public class IcebergOverwritePartitionsStep implements CommitStep {
+  private final String destTableIdStr;
+  private final Properties properties;
+  private final byte[] serializedDataFiles;
+  private final String partitionColName;
+  private final String partitionValue;
+  public static final String OVERWRITE_PARTITIONS_RETRYER_CONFIG_PREFIX = 
IcebergDatasetFinder.ICEBERG_DATASET_PREFIX +
+      ".catalog.overwrite.partitions.retries";
+  private static final Config RETRYER_FALLBACK_CONFIG = 
ConfigFactory.parseMap(ImmutableMap.of(
+      RETRY_INTERVAL_MS, TimeUnit.SECONDS.toMillis(3L),
+      RETRY_TIMES, 3,
+      RETRY_TYPE, RetryType.FIXED_ATTEMPT.name()));
+
+  /**
+   * Constructs an {@code IcebergReplacePartitionsStep} with the specified 
parameters.
+   *
+   * @param destTableIdStr the identifier of the destination table as a string
+   * @param serializedDataFiles [from List<DataFiles>] the serialized data 
files to be used for replacing partitions
+   * @param properties the properties containing configuration
+   */
+  public IcebergOverwritePartitionsStep(String destTableIdStr, String 
partitionColName, String partitionValue, byte[] serializedDataFiles, Properties 
properties) {
+    this.destTableIdStr = destTableIdStr;
+    this.partitionColName = partitionColName;
+    this.partitionValue = partitionValue;
+    this.serializedDataFiles = serializedDataFiles;
+    this.properties = properties;
+  }
+
+  @Override
+  public boolean isCompleted() {
+    return false;
+  }
+
+  /**
+   * Executes the partition replacement in the destination Iceberg table.
+   * Also, have retry mechanism as done in {@link 
IcebergRegisterStep#execute()}
+   *
+   * @throws IOException if an I/O error occurs during execution
+   */
+  @Override
+  public void execute() throws IOException {
+    IcebergTable destTable = 
createDestinationCatalog().openTable(TableIdentifier.parse(this.destTableIdStr));
+    List<DataFile> dataFiles = 
SerializationUtil.deserializeFromBytes(this.serializedDataFiles);
+    try {
+      log.info("Overwriting Data files of partition {} with value {} for 
destination table : {} ",
+          this.partitionColName,
+          this.partitionValue,
+          this.destTableIdStr
+      );

Review Comment:
   let's log `dataFiles.size()`.
   
   since there could be quite a few, we shouldn't log every path, but IME it 
can help to at least log one or two.  e.g. in case something about the content 
leads to failure/corruption (like they turn out to be relative paths or are 
actually URLs that have a hostname along w/ the path)



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/predicates/IcebergPartitionFilterPredicateUtil.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg.predicates;
+
+import java.util.List;
+
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.TableMetadata;
+
+/**
+ * Utility class for creating and managing partition filter predicates for 
Iceberg tables.
+ * <p>
+ * This class provides methods to retrieve the index of a partition column in 
the table metadata
+ * and ensures that the partition transform is supported.
+ * </p>
+ * <p>
+ * Note: This class is not meant to be instantiated.
+ * </p>
+ */
+public class IcebergPartitionFilterPredicateUtil {
+  private IcebergPartitionFilterPredicateUtil() {
+  }
+
+  /**
+   * Retrieves the index of the partition column from the partition spec in 
the table metadata.
+   *
+   * @param partitionColumnName the name of the partition column to find
+   * @param tableMetadata the metadata of the Iceberg table
+   * @param supportedTransforms a list of supported partition transforms
+   * @return the index of the partition column if found, otherwise -1
+   * @throws IllegalArgumentException if the partition transform is not 
supported
+   */
+  public static int getPartitionColumnIndex(

Review Comment:
   this single `static` seems closely related enough to 
`IcebergMatchesAnyPropNamePartitionFilterPredicate` that it could reasonably 
live there as a `public static` (eliminating the need for an additional 
separate class).



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergPartitionDataset.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.ImmutableList;
+
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.data.management.copy.CopyConfiguration;
+import org.apache.gobblin.data.management.copy.CopyEntity;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
+import org.apache.gobblin.data.management.copy.CopyableDataset;
+import org.apache.gobblin.util.measurement.GrowthMilestoneTracker;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergMatchesAnyPropNamePartitionFilterPredicate;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergPartitionFilterPredicateUtil;
+
+/**
+ * Iceberg Partition dataset implementing {@link CopyableDataset}
+ * <p>
+ * This class extends {@link IcebergDataset} and provides functionality to 
filter partitions
+ * and generate copy entities for partition based data movement.
+ * </p>
+ */
+@Slf4j
+public class IcebergPartitionDataset extends IcebergDataset {
+  private static final List<String> supportedTransforms = 
ImmutableList.of("identity", "truncate");
+  private final Predicate<StructLike> partitionFilterPredicate;
+  private final Map<Path, Path> srcPathToDestPath;
+  private final String partitionColumnName;
+  private final String partitionColValue;
+
+  public IcebergPartitionDataset(IcebergTable srcIcebergTable, IcebergTable 
destIcebergTable, Properties properties,
+      FileSystem sourceFs, boolean shouldIncludeMetadataPath, String 
partitionColumnName, String partitionColValue)
+      throws IcebergTable.TableNotFoundException {
+    super(srcIcebergTable, destIcebergTable, properties, sourceFs, 
shouldIncludeMetadataPath);
+    this.partitionColumnName = partitionColumnName;
+    this.partitionColValue = partitionColValue;
+    this.partitionFilterPredicate = createPartitionFilterPredicate();
+    this.srcPathToDestPath = new HashMap<>();
+  }
+
+  private Predicate<StructLike> createPartitionFilterPredicate() throws 
IcebergTable.TableNotFoundException {
+    //TODO: Refactor it later using factory or other way to support different 
types of filter predicate
+    // Also take into consideration creation of Expression Filter to be used 
in overwrite api
+    TableMetadata srcTableMetadata = 
getSrcIcebergTable().accessTableMetadata();
+    int partitionColumnIndex = 
IcebergPartitionFilterPredicateUtil.getPartitionColumnIndex(
+        this.partitionColumnName,
+        srcTableMetadata,
+        supportedTransforms
+    );
+    Preconditions.checkArgument(partitionColumnIndex >= 0, String.format(
+        "Partition column %s not found in table %s",
+        this.partitionColumnName, this.getFileSetId()));
+    return new 
IcebergMatchesAnyPropNamePartitionFilterPredicate(partitionColumnIndex, 
this.partitionColValue);
+  }
+
+  /**
+   * Represents the destination file paths and the corresponding file status 
in source file system.
+   * These both properties are used in creating {@link CopyEntity}
+   */
+  @Data
+  protected static final class FilePathsWithStatus {
+    private final Path destPath;
+    private final FileStatus srcFileStatus;
+  }
+
+  /**
+   * Generates copy entities for partition based data movement.
+   * It finds files specific to the partition and create destination data 
files based on the source data files.
+   * Also updates the destination data files with destination table write data 
location and add UUID to the file path
+   * to avoid conflicts.
+   *
+   * @param targetFs the target file system
+   * @param copyConfig the copy configuration
+   * @return a collection of copy entities
+   * @throws IOException if an I/O error occurs
+   */
+  @Override
+  Collection<CopyEntity> generateCopyEntities(FileSystem targetFs, 
CopyConfiguration copyConfig) throws IOException {
+    String fileSet = this.getFileSetId();
+    List<CopyEntity> copyEntities = Lists.newArrayList();
+    IcebergTable srcIcebergTable = getSrcIcebergTable();
+    List<DataFile> srcDataFiles = 
srcIcebergTable.getPartitionSpecificDataFiles(this.partitionFilterPredicate);
+    List<DataFile> destDataFiles = getDestDataFiles(srcDataFiles);
+    Configuration defaultHadoopConfiguration = new Configuration();
+
+    for (FilePathsWithStatus filePathsWithStatus : 
getFilePathsStatus(this.sourceFs)) {
+      Path destPath = filePathsWithStatus.getDestPath();
+      FileStatus srcFileStatus = filePathsWithStatus.getSrcFileStatus();
+      FileSystem actualSourceFs = 
getSourceFileSystemFromFileStatus(srcFileStatus, defaultHadoopConfiguration);
+
+      CopyableFile fileEntity = CopyableFile.fromOriginAndDestination(
+              actualSourceFs, srcFileStatus, targetFs.makeQualified(destPath), 
copyConfig)
+          .fileSet(fileSet)
+          .datasetOutputPath(targetFs.getUri().getPath())
+          .build();
+
+      fileEntity.setSourceData(getSourceDataset(this.sourceFs));
+      fileEntity.setDestinationData(getDestinationDataset(targetFs));
+      copyEntities.add(fileEntity);
+    }
+
+    // Adding this check to avoid adding post publish step when there are no 
files to copy.
+    if (CollectionUtils.isNotEmpty(destDataFiles)) {
+      copyEntities.add(createOverwritePostPublishStep(destDataFiles));
+    }
+
+    log.info("~{}~ generated {} copy--entities", fileSet, copyEntities.size());
+    return copyEntities;
+  }
+
+  private List<DataFile> getDestDataFiles(List<DataFile> srcDataFiles) throws 
IcebergTable.TableNotFoundException {
+    List<DataFile> destDataFiles = new ArrayList<>();
+    if (srcDataFiles.isEmpty()) {
+      log.warn("No data files found for partition col : {} with partition 
value : {} in source table : {}",
+          this.partitionColumnName, this.partitionColValue, 
this.getFileSetId());
+      return destDataFiles;
+    }
+    TableMetadata srcTableMetadata = 
getSrcIcebergTable().accessTableMetadata();
+    TableMetadata destTableMetadata = 
getDestIcebergTable().accessTableMetadata();
+    PartitionSpec partitionSpec = destTableMetadata.spec();
+    String srcWriteDataLocation = 
srcTableMetadata.property(TableProperties.WRITE_DATA_LOCATION, "");
+    String destWriteDataLocation = 
destTableMetadata.property(TableProperties.WRITE_DATA_LOCATION, "");
+    if (StringUtils.isEmpty(srcWriteDataLocation) || 
StringUtils.isEmpty(destWriteDataLocation)) {
+      log.warn(
+          "Either source or destination table does not have write data 
location : source table write data location : {} , destination table write data 
location : {}",
+          srcWriteDataLocation,
+          destWriteDataLocation
+      );
+    }
+    // tableMetadata.property(TableProperties.WRITE_DATA_LOCATION, "") returns 
null if the property is not set and
+    // doesn't respect passed default value, so to avoid NPE in .replace() we 
are setting it to empty string.
+    String prefixToBeReplaced = (srcWriteDataLocation != null) ? 
srcWriteDataLocation : "";
+    String prefixToReplaceWith = (destWriteDataLocation != null) ? 
destWriteDataLocation : "";
+    GrowthMilestoneTracker growthMilestoneTracker = new 
GrowthMilestoneTracker();
+    srcDataFiles.forEach(dataFile -> {
+      String srcFilePath = dataFile.path().toString();
+      Path updatedDestFilePath = relocateDestPath(srcFilePath, 
prefixToBeReplaced, prefixToReplaceWith);
+      destDataFiles.add(DataFiles.builder(partitionSpec)
+          .copy(dataFile)
+          .withPath(updatedDestFilePath.toString())
+          .build());
+      // Store the mapping of srcPath to destPath to be used in creating list 
of src file status to dest path
+      log.debug("Path changed from Src : {} to Dest : {}", srcFilePath, 
updatedDestFilePath);
+      srcPathToDestPath.put(new Path(srcFilePath), updatedDestFilePath);
+      if (growthMilestoneTracker.isAnotherMilestone(destDataFiles.size())) {
+        log.info("Generated {} destination data files", destDataFiles.size());
+      }
+    });
+    log.info("Generated {} destination data files", destDataFiles.size());

Review Comment:
   as mentioned earlier, let's keep the logging convention (that names the 
table) from `IcebergDataset`, specifically:
   ```
   log.info("~{}~ doing ...", this.getFileSetId() ...)
   ```



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergPartitionDataset.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.ImmutableList;
+
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.data.management.copy.CopyConfiguration;
+import org.apache.gobblin.data.management.copy.CopyEntity;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
+import org.apache.gobblin.data.management.copy.CopyableDataset;
+import org.apache.gobblin.util.measurement.GrowthMilestoneTracker;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergMatchesAnyPropNamePartitionFilterPredicate;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergPartitionFilterPredicateUtil;
+
+/**
+ * Iceberg Partition dataset implementing {@link CopyableDataset}
+ * <p>
+ * This class extends {@link IcebergDataset} and provides functionality to 
filter partitions
+ * and generate copy entities for partition based data movement.
+ * </p>
+ */
+@Slf4j
+public class IcebergPartitionDataset extends IcebergDataset {
+  private static final List<String> supportedTransforms = 
ImmutableList.of("identity", "truncate");
+  private final Predicate<StructLike> partitionFilterPredicate;
+  private final Map<Path, Path> srcPathToDestPath;

Review Comment:
   I didn't catch this before, but as a general rule, we prefer to avoid 
stateful classes, which hold opportunity for corruption and require care for 
thread-safety.
   
   certainly statefulness can be necessary, but I don't see that here.  a 
`private` method may of course return a result to later pass on to a subsequent 
`private` method, via local reference.  there's no reason to rely on the 
instance itself to ferry temporary/intermediate forms of a computation.
   
   `srcPathToDestPath` seems purely an ephemeral way to tunnel data between 
`private` methods which all work together during a single invocation of a 
`public` method.  please describe if you see it differently



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergTable.java:
##########
@@ -217,4 +227,59 @@ protected void registerIcebergTable(TableMetadata 
srcMetadata, TableMetadata dst
       this.tableOps.commit(dstMetadata, srcMetadata);
     }
   }
+
+  /**
+   * Retrieves a list of data files from the current snapshot that match the 
specified partition filter predicate.
+   *
+   * @param icebergPartitionFilterPredicate the predicate to filter partitions
+   * @return a list of data files that match the partition filter predicate
+   * @throws IOException if an I/O error occurs while accessing the table 
metadata or reading manifest files
+   */
+  public List<DataFile> getPartitionSpecificDataFiles(Predicate<StructLike> 
icebergPartitionFilterPredicate) throws IOException {
+    TableMetadata tableMetadata = accessTableMetadata();
+    Snapshot currentSnapshot = tableMetadata.currentSnapshot();
+    log.info("Starting to copy data files from snapshot: {}", 
currentSnapshot.snapshotId());
+    //TODO: Add support for deleteManifests as well later
+    // Currently supporting dataManifests only
+    List<ManifestFile> dataManifestFiles = 
currentSnapshot.dataManifests(this.tableOps.io());
+    List<DataFile> dataFileList = new ArrayList<>();
+    for (ManifestFile manifestFile : dataManifestFiles) {
+      try (ManifestReader<DataFile> manifestReader = 
ManifestFiles.read(manifestFile, this.tableOps.io());
+          CloseableIterator<DataFile> dataFiles = manifestReader.iterator()) {
+        dataFiles.forEachRemaining(dataFile -> {
+          if (icebergPartitionFilterPredicate.test(dataFile.partition())) {
+            dataFileList.add(dataFile.copy());
+          }
+        });
+      } catch (IOException e) {
+        log.warn("Failed to read manifest file: {} " , manifestFile.path(), e);
+      }
+    }
+    log.info("Found {} data files to copy", dataFileList.size());

Review Comment:
   since these iterative reads can take quite a while, let's ensure logging for 
meaningful observability, even including machine parsing.
   
   would you please try to align these msgs w/ the ones just above in the 
`getIncrementalSnapshotInfosIterator()` method?  e.g.
   ```
         log.info("~{}~ before snapshot '{}' - '{}' total known iceberg paths",
             tableId, snapshotInfo.getSnapshotId(), knownFilePaths.size());
   ```
   (that could subsume both "starting..." and "found...")



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/predicates/IcebergPartitionFilterPredicateUtil.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg.predicates;
+
+import java.util.List;
+
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.TableMetadata;
+
+/**
+ * Utility class for creating and managing partition filter predicates for 
Iceberg tables.
+ * <p>
+ * This class provides methods to retrieve the index of a partition column in 
the table metadata
+ * and ensures that the partition transform is supported.
+ * </p>
+ * <p>
+ * Note: This class is not meant to be instantiated.
+ * </p>
+ */
+public class IcebergPartitionFilterPredicateUtil {
+  private IcebergPartitionFilterPredicateUtil() {
+  }
+
+  /**
+   * Retrieves the index of the partition column from the partition spec in 
the table metadata.
+   *
+   * @param partitionColumnName the name of the partition column to find
+   * @param tableMetadata the metadata of the Iceberg table
+   * @param supportedTransforms a list of supported partition transforms
+   * @return the index of the partition column if found, otherwise -1
+   * @throws IllegalArgumentException if the partition transform is not 
supported
+   */
+  public static int getPartitionColumnIndex(
+      String partitionColumnName,
+      TableMetadata tableMetadata,
+      List<String> supportedTransforms
+  ) {
+    List<PartitionField> partitionFields = tableMetadata.spec().fields();
+    for (int idx = 0; idx < partitionFields.size(); idx++) {
+      PartitionField partitionField = partitionFields.get(idx);
+      if (partitionField.name().equals(partitionColumnName)) {
+        String transform = partitionField.transform().toString().toLowerCase();
+        if (!supportedTransforms.contains(transform)) {
+          throw new IllegalArgumentException(
+              String.format(" For ~{%s:%d}~ Partition transform %s is not 
supported. Supported transforms are %s",
+                  partitionColumnName,
+                  idx,
+                  transform,
+                  supportedTransforms));
+        }
+        return idx;
+      }
+    }
+    return -1;

Review Comment:
   the interface would be clearer to return `Optional<Integer>` rather than 
this `-1` sentinel value a caller could easily forget to check for



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/predicates/IcebergPartitionFilterPredicateUtil.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg.predicates;
+
+import java.util.List;
+
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.TableMetadata;
+
+/**
+ * Utility class for creating and managing partition filter predicates for 
Iceberg tables.
+ * <p>
+ * This class provides methods to retrieve the index of a partition column in 
the table metadata
+ * and ensures that the partition transform is supported.
+ * </p>
+ * <p>
+ * Note: This class is not meant to be instantiated.
+ * </p>
+ */
+public class IcebergPartitionFilterPredicateUtil {
+  private IcebergPartitionFilterPredicateUtil() {
+  }
+
+  /**
+   * Retrieves the index of the partition column from the partition spec in 
the table metadata.
+   *
+   * @param partitionColumnName the name of the partition column to find
+   * @param tableMetadata the metadata of the Iceberg table
+   * @param supportedTransforms a list of supported partition transforms
+   * @return the index of the partition column if found, otherwise -1
+   * @throws IllegalArgumentException if the partition transform is not 
supported
+   */
+  public static int getPartitionColumnIndex(
+      String partitionColumnName,
+      TableMetadata tableMetadata,
+      List<String> supportedTransforms
+  ) {
+    List<PartitionField> partitionFields = tableMetadata.spec().fields();
+    for (int idx = 0; idx < partitionFields.size(); idx++) {
+      PartitionField partitionField = partitionFields.get(idx);
+      if (partitionField.name().equals(partitionColumnName)) {
+        String transform = partitionField.transform().toString().toLowerCase();
+        if (!supportedTransforms.contains(transform)) {
+          throw new IllegalArgumentException(

Review Comment:
   this doesn't seem like an illegal arg situation, since what the caller 
provided is their choice, but here we're throwing because we found the table 
did not match that



##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergPartitionDataset.java:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.gobblin.data.management.copy.iceberg;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.util.SerializationUtil;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.base.Preconditions;
+
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.data.management.copy.CopyConfiguration;
+import org.apache.gobblin.data.management.copy.CopyEntity;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
+import org.apache.gobblin.data.management.copy.CopyableDataset;
+import 
org.apache.gobblin.data.management.copy.iceberg.predicates.IcebergPartitionFilterPredicateFactory;
+
+/**
+ * Iceberg Partition dataset implementing {@link CopyableDataset}
+ * <p>
+ * This class extends {@link IcebergDataset} and provides functionality to 
filter partitions
+ * and generate copy entities for partition based data movement.
+ * </p>
+ */
+@Slf4j
+public class IcebergPartitionDataset extends IcebergDataset {
+
+  private static final String ICEBERG_PARTITION_NAME_KEY = "partition.name";
+  private final Predicate<StructLike> partitionFilterPredicate;
+
+  public IcebergPartitionDataset(IcebergTable srcIcebergTable, IcebergTable 
destIcebergTable, Properties properties,
+      FileSystem sourceFs, boolean shouldIncludeMetadataPath) throws 
IcebergTable.TableNotFoundException {
+    super(srcIcebergTable, destIcebergTable, properties, sourceFs, 
shouldIncludeMetadataPath);
+
+    String partitionColumnName =
+        IcebergDatasetFinder.getLocationQualifiedProperty(properties, 
IcebergDatasetFinder.CatalogLocation.SOURCE,
+            ICEBERG_PARTITION_NAME_KEY);
+    Preconditions.checkArgument(StringUtils.isNotEmpty(partitionColumnName),
+        "Partition column name cannot be empty");
+
+    TableMetadata srcTableMetadata = 
getSrcIcebergTable().accessTableMetadata();
+    this.partitionFilterPredicate = 
IcebergPartitionFilterPredicateFactory.getFilterPredicate(partitionColumnName,
+        srcTableMetadata, properties);
+  }
+
+  /**
+   * Represents the destination file paths and the corresponding file status 
in source file system.
+   * These both properties are used in creating {@link CopyEntity}
+   */
+  @Data
+  protected static final class FilePathsWithStatus {
+    private final Path destPath;
+    private final FileStatus srcFileStatus;
+  }

Review Comment:
   AFAICT the `Map<Path, FileStatus>` in `IcebergDataset` just treats the key 
as a path, not specifically src vs. dest.
   
   so that still seems worth following here, rather than inventing a new 
one-off type to convey equivalent info.
   
   I'll leave the choice to you, but consistency is my recommendation.



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

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

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

Reply via email to