[
https://issues.apache.org/jira/browse/GOBBLIN-2159?focusedWorklogId=936274&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-936274
]
ASF GitHub Bot logged work on GOBBLIN-2159:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 25/Sep/24 04:17
Start Date: 25/Sep/24 04:17
Worklog Time Spent: 10m
Work Description: Blazer-007 commented on code in PR #4058:
URL: https://github.com/apache/gobblin/pull/4058#discussion_r1774390292
##########
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;
+ }
+
+ /**
+ * 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(srcDataFiles, destDataFiles, 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();
Review Comment:
In the IcebergDataset the path of tables are exactly since table UUID are
same on source and destination here it can be different, so copying permissions
atleast in first draft is not necessary I believe.
Even if there is need that we need to make sure ancestor path, parent path
are ones we want, that's why I have removed it for now.
Issue Time Tracking
-------------------
Worklog Id: (was: 936274)
Time Spent: 1h (was: 50m)
> Support Partition Based Copy in Iceberg Distcp
> ----------------------------------------------
>
> Key: GOBBLIN-2159
> URL: https://issues.apache.org/jira/browse/GOBBLIN-2159
> Project: Apache Gobblin
> Issue Type: Task
> Reporter: Vivek Rai
> Priority: Major
> Time Spent: 1h
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)