[
https://issues.apache.org/jira/browse/GOBBLIN-1709?focusedWorklogId=809943&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-809943
]
ASF GitHub Bot logged work on GOBBLIN-1709:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 19/Sep/22 06:18
Start Date: 19/Sep/22 06:18
Worklog Time Spent: 10m
Work Description: meethngala commented on code in PR #3560:
URL: https://github.com/apache/gobblin/pull/3560#discussion_r973893265
##########
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/iceberg/IcebergDataset.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.net.URI;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+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.jetbrains.annotations.NotNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import lombok.Data;
+import lombok.Getter;
+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.CopyableDataset;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import
org.apache.gobblin.data.management.copy.prioritization.PrioritizedCopyableDataset;
+import org.apache.gobblin.data.management.partition.FileSet;
+import org.apache.gobblin.dataset.DatasetConstants;
+import org.apache.gobblin.dataset.DatasetDescriptor;
+import org.apache.gobblin.util.request_allocation.PushDownRequestor;
+
+/**
+ * Iceberg dataset implementing {@link CopyableDataset}.
+ */
+@Slf4j
+@Getter
+public class IcebergDataset implements PrioritizedCopyableDataset {
+ private final String dbName;
+ private final String inputTableName;
+ private IcebergTable icebergTable;
+ protected Properties properties;
+ protected FileSystem fs;
+
+ private Optional<String> sourceMetastoreURI;
+ private Optional<String> targetMetastoreURI;
+
+ /** Target metastore URI */
+ public static final String TARGET_METASTORE_URI_KEY =
+ IcebergDatasetFinder.ICEBERG_DATASET_PREFIX +
".copy.target.metastore.uri";
+ /** Target database name */
+ public static final String TARGET_DATABASE_KEY =
IcebergDatasetFinder.ICEBERG_DATASET_PREFIX + ".copy.target.database";
+
+ public IcebergDataset(String db, String table, IcebergTable icebergTbl,
Properties properties, FileSystem fs) {
+ this.dbName = db;
+ this.inputTableName = table;
+ this.icebergTable = icebergTbl;
+ this.properties = properties;
+ this.fs = fs;
+ this.sourceMetastoreURI =
+
Optional.fromNullable(this.properties.getProperty(IcebergDatasetFinder.ICEBERG_HIVE_CATALOG_METASTORE_URI_KEY));
+ this.targetMetastoreURI =
+
Optional.fromNullable(this.properties.getProperty(TARGET_METASTORE_URI_KEY));
+ }
+
+ /**
+ * Represents a source {@link FileStatus} and a {@link Path} destination.
+ */
+ @Data
+ private static class SourceAndDestination {
+ private final FileStatus source;
+ private final Path destination;
+ }
+
+ @Override
+ public String datasetURN() {
+ // TODO: verify!
+ return this.dbName + "." + this.inputTableName;
+ }
+
+ /**
+ * Finds all files read by the table and generates CopyableFiles.
+ * For the specific semantics see {@link #getCopyEntities}.
+ */
+ @Override
+ public Iterator<FileSet<CopyEntity>> getFileSetIterator(FileSystem targetFs,
CopyConfiguration configuration) {
+ return getCopyEntities(configuration);
+ }
+ /**
+ * Finds all files read by the table and generates CopyableFiles.
+ * For the specific semantics see {@link #getCopyEntities}.
+ */
+ @Override
+ public Iterator<FileSet<CopyEntity>> getFileSetIterator(FileSystem targetFs,
CopyConfiguration configuration,
+ Comparator<FileSet<CopyEntity>> prioritizer,
PushDownRequestor<FileSet<CopyEntity>> requestor) {
+ // TODO: Implement PushDownRequestor and priority based copy entity
iteration
+ return getCopyEntities(configuration);
+ }
+
+ /**
+ * Finds all files read by the table and generates {@link CopyEntity}s for
duplicating the table.
+ */
+ Iterator<FileSet<CopyEntity>> getCopyEntities(CopyConfiguration
configuration) {
+ FileSet<CopyEntity> fileSet = new
IcebergTableFileSet(this.getInputTableName(), this, configuration);
+ return Iterators.singletonIterator(fileSet); }
+
+ /**
+ * Finds all files read by the table file set and generates {@link
CopyEntity}s for duplicating the table.
+ */
+ @VisibleForTesting
+ Collection<CopyEntity> generateCopyEntities(CopyConfiguration configuration)
throws IOException {
+ String fileSet = this.getInputTableName();
+ List<CopyEntity> copyEntities = Lists.newArrayList();
+ Map<Path, FileStatus> mapOfPathsToCopy = getFilePaths();
+ log.info("Found {} to be copied over to the destination",
mapOfPathsToCopy.size());
+
+
+ for (CopyableFile.Builder builder :
getCopyableFilesFromPaths(mapOfPathsToCopy, configuration)) {
+ CopyableFile fileEntity =
+
builder.fileSet(fileSet).datasetOutputPath(this.fs.getUri().getPath()).build();
+ fileEntity.setSourceData(getSourceDataset());
+ fileEntity.setDestinationData(getDestinationDataset());
+ log.info("Adding file path: {} to the collection of copy entities and
copying to destination: {}",
Review Comment:
Honestly, I foresee it to become noisy and hence decided to just keep the
overall count of copy entities as part of the log statement in the end which
you have pointed out above. The log level as `debug` might be unnecessary for
this implementation... and we can always add it later and activate it on a
case-by case basis... that's my take on it!
Issue Time Tracking
-------------------
Worklog Id: (was: 809943)
Time Spent: 10h (was: 9h 50m)
> Create work units for Hive Catalog based Iceberg Datasets to support Distcp
> for Iceberg
> ---------------------------------------------------------------------------------------
>
> Key: GOBBLIN-1709
> URL: https://issues.apache.org/jira/browse/GOBBLIN-1709
> Project: Apache Gobblin
> Issue Type: New Feature
> Components: distcp-ng
> Reporter: Meeth Gala
> Assignee: Issac Buenrostro
> Priority: Major
> Time Spent: 10h
> Remaining Estimate: 0h
>
> We want to support Distcp for Iceberg based datasets.
> As a pilot, we are starting with Hive Catalog and will expand the
> functionality to cover all Iceberg based datasets.
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)