You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/01/01 00:47:48 UTC

[GitHub] [hudi] prashantwason commented on a change in pull request #2398: [HUDI-842] Implementation of HUDI RFC-15.

prashantwason commented on a change in pull request #2398:
URL: https://github.com/apache/hudi/pull/2398#discussion_r550706532



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
##########
@@ -481,6 +481,11 @@ private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient met
     throw new HoodieException("FileGroupId " + fgId + " not in pending compaction");
   }
 
+  @Override
+  protected void initWrapperFSMetrics() {

Review comment:
       If this is empty, is this required as the base class already has this implementation.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metrics/SparkHoodieBackedTableMetadataWriter.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.metrics;

Review comment:
       package seems wrong from the file name.

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -88,6 +88,11 @@
   private final HoodieTable<T, I, K, O> table;
   private final HoodieTableMetaClient metaClient;
 
+  /*

Review comment:
       Dead code?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
##########
@@ -240,12 +252,14 @@ public static void processFiles(FileSystem fs, String basePathStr, Function<File
     }
   }
 
-  public static List<String> getAllPartitionPaths(FileSystem fs, String basePathStr, boolean assumeDatePartitioning)
-      throws IOException {
+  public static List<String> getAllPartitionPaths(FileSystem fs, String basePathStr, boolean useFileListingFromMetadata, boolean verifyListings,
+                                                  boolean assumeDatePartitioning) throws IOException {
     if (assumeDatePartitioning) {

Review comment:
       this should be within another if block:
   
   if(useFileListingFromMetadata) {
   }
   

##########
File path: hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java
##########
@@ -0,0 +1,293 @@
+
+/*
+ * 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.metadata;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.avro.model.HoodieMetadataRecord;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hudi.exception.HoodieMetadataException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public abstract class BaseTableMetadata implements HoodieTableMetadata {
+
+  private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class);
+
+  static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024;
+  static final int BUFFER_SIZE = 10 * 1024 * 1024;
+
+  protected final SerializableConfiguration hadoopConf;
+  protected final String datasetBasePath;
+  protected boolean enabled;
+  protected final Option<HoodieMetadataMetrics> metrics;
+
+  private final boolean validateLookups;
+  private final boolean assumeDatePartitioning;
+
+  // Directory used for Spillable Map when merging records
+  protected final String spillableMapDirectory;
+  private transient HoodieMetadataMergedInstantRecordScanner timelineRecordScanner;
+
+  protected BaseTableMetadata(Configuration hadoopConf, String datasetBasePath, String spillableMapDirectory,
+                              boolean enabled, boolean validateLookups, boolean enableMetrics,
+                              boolean assumeDatePartitioning) {
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+    this.datasetBasePath = datasetBasePath;
+    this.spillableMapDirectory = spillableMapDirectory;
+
+    this.enabled = enabled;
+    this.validateLookups = validateLookups;
+    this.assumeDatePartitioning = assumeDatePartitioning;
+
+    if (enableMetrics) {
+      this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata")));
+    } else {
+      this.metrics = Option.empty();
+    }
+  }
+
+  /**
+   * Return the list of partitions in the dataset.
+   *
+   * If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of
+   * partitions is retrieved directly from the underlying {@code FileSystem}.
+   *
+   * On any errors retrieving the listing from the metadata, defaults to using the file system listings.
+   *
+   */
+  @Override
+  public List<String> getAllPartitionPaths() throws IOException {
+    if (enabled) {
+      try {
+        return fetchAllPartitionPaths();
+      } catch (Exception e) {
+        LOG.error("Failed to retrieve list of partition from metadata", e);
+      }
+    }
+    return new FileSystemBackedTableMetadata(hadoopConf, datasetBasePath, assumeDatePartitioning).getAllPartitionPaths();
+  }
+
+  /**
+   * Return the list of files in a partition.
+   *
+   * If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of
+   * partitions is retrieved directly from the underlying {@code FileSystem}.
+   *
+   * On any errors retrieving the listing from the metadata, defaults to using the file system listings.
+   *
+   * @param partitionPath The absolute path of the partition to list
+   */
+  @Override
+  public FileStatus[] getAllFilesInPartition(Path partitionPath)
+      throws IOException {
+    if (enabled) {
+      try {
+        return fetchAllFilesInPartition(partitionPath);
+      } catch (Exception e) {
+        LOG.error("Failed to retrieve files in partition " + partitionPath + " from metadata", e);
+      }
+    }
+
+    return FSUtils.getFs(partitionPath.toString(), hadoopConf.get()).listStatus(partitionPath);

Review comment:
       This can be from FileSystemBackedTableMetadata.getAllFilesInPartition(...) for symmetry with above method.




----------------------------------------------------------------
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.

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