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 2022/03/31 09:01:12 UTC

[GitHub] [hudi] xushiyan commented on a change in pull request #5153: [HUDI-3020] Utility to create manifest file

xushiyan commented on a change in pull request #5153:
URL: https://github.com/apache/hudi/pull/5153#discussion_r839311861



##########
File path: hudi-sync/hudi-sync-common/pom.xml
##########
@@ -104,6 +104,14 @@
       <artifactId>junit-platform-commons</artifactId>
       <scope>test</scope>
     </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>

Review comment:
       i don't think this is needed. hudi-common is available every module

##########
File path: hudi-common/src/main/java/org/apache/hudi/metadata/ManifestFileUtil.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+
+public class ManifestFileUtil {
+  private static final long serialVersionUID = 1L;

Review comment:
       do you mean to make this Serializeable?

##########
File path: hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileUtil.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.sync.common.util;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+
+public class ManifestFileUtil {
+  private static final long serialVersionUID = 1L;

Review comment:
       thought you wanted this Serializeable?

##########
File path: hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileUtil.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.sync.common.util;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+
+public class ManifestFileUtil {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(ManifestFileUtil.class);
+  private static final String MANIFEST_FOLDER_NAME = "manifest";
+  private static final String MANIFEST_FILE_NAME = "latest-snapshot.csv";
+  private static final String DELIMITER = "\n";
+  private SerializableConfiguration hadoopConf;
+  private String basePath;
+  private transient HoodieLocalEngineContext engineContext;
+  private HoodieTableMetaClient metaClient;
+
+  private ManifestFileUtil(Configuration conf, String basePath) {
+    this.hadoopConf = new SerializableConfiguration(conf);
+    this.basePath = basePath;
+    this.engineContext = new HoodieLocalEngineContext(conf);
+    this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
+  }
+
+  public ManifestFileUtil() {
+  }

Review comment:
       do you need this? thought you want to enforce builder for creating it.

##########
File path: hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileUtil.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.sync.common.util;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+
+public class ManifestFileUtil {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(ManifestFileUtil.class);
+  private static final String MANIFEST_FOLDER_NAME = "manifest";
+  private static final String MANIFEST_FILE_NAME = "latest-snapshot.csv";
+  private static final String DELIMITER = "\n";
+  private SerializableConfiguration hadoopConf;
+  private String basePath;
+  private transient HoodieLocalEngineContext engineContext;
+  private HoodieTableMetaClient metaClient;
+
+  private ManifestFileUtil(Configuration conf, String basePath) {
+    this.hadoopConf = new SerializableConfiguration(conf);
+    this.basePath = basePath;
+    this.engineContext = new HoodieLocalEngineContext(conf);
+    this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
+  }
+
+  public ManifestFileUtil() {
+  }
+
+  public synchronized void writeManifestFile() {
+    try {
+      Path manifestFilePath = new Path(getManifestFolder(), MANIFEST_FILE_NAME);
+      Option<byte[]> content = Option.of(fetchLatestBaseFilesForAllPartitions().collect(Collectors.joining(DELIMITER)).getBytes());
+      FileIOUtils.createFileInPath(metaClient.getFs(), manifestFilePath, content);
+    } catch (Exception e) {
+      String msg = "Error writing manifest file";
+      LOG.error(msg, e);
+      throw new HoodieException(msg, e);
+    }
+  }
+
+  public Stream<String> fetchLatestBaseFilesForAllPartitions() {
+    try {
+      HoodieMetadataConfig metadataConfig = buildMetadataConfig(hadoopConf.get());
+      HoodieMetadataFileSystemView fsView = new HoodieMetadataFileSystemView(engineContext, metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), metadataConfig);
+
+      List<String> partitions = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath);
+
+      return partitions.parallelStream().flatMap(p -> fsView.getLatestBaseFiles(p).map(HoodieBaseFile::getFileName));
+    } catch (Exception e) {
+      String msg = "Error checking path :" + basePath;
+      LOG.error(msg, e);
+      throw new HoodieException(msg, e);
+    }
+  }
+
+  private static HoodieMetadataConfig buildMetadataConfig(Configuration conf) {
+    return HoodieMetadataConfig.newBuilder()
+        .enable(conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS))
+        .build();
+  }
+
+  /**
+   * @return Manifest File folder
+   */
+  public String getManifestFolder() {
+    return metaClient.getMetaPath() + Path.SEPARATOR + MANIFEST_FOLDER_NAME;
+  }

Review comment:
       since you need hadoop Path anyway, suggest create Path instead of concat strings

##########
File path: hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileUtil.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.sync.common.util;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+
+public class ManifestFileUtil {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(ManifestFileUtil.class);
+  private static final String MANIFEST_FOLDER_NAME = "manifest";
+  private static final String MANIFEST_FILE_NAME = "latest-snapshot.csv";
+  private static final String DELIMITER = "\n";
+  private SerializableConfiguration hadoopConf;
+  private String basePath;
+  private transient HoodieLocalEngineContext engineContext;
+  private HoodieTableMetaClient metaClient;
+
+  private ManifestFileUtil(Configuration conf, String basePath) {
+    this.hadoopConf = new SerializableConfiguration(conf);
+    this.basePath = basePath;
+    this.engineContext = new HoodieLocalEngineContext(conf);
+    this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
+  }
+
+  public ManifestFileUtil() {
+  }
+
+  public synchronized void writeManifestFile() {
+    try {
+      Path manifestFilePath = new Path(getManifestFolder(), MANIFEST_FILE_NAME);
+      Option<byte[]> content = Option.of(fetchLatestBaseFilesForAllPartitions().collect(Collectors.joining(DELIMITER)).getBytes());
+      FileIOUtils.createFileInPath(metaClient.getFs(), manifestFilePath, content);
+    } catch (Exception e) {
+      String msg = "Error writing manifest file";
+      LOG.error(msg, e);
+      throw new HoodieException(msg, e);
+    }
+  }
+
+  public Stream<String> fetchLatestBaseFilesForAllPartitions() {
+    try {
+      HoodieMetadataConfig metadataConfig = buildMetadataConfig(hadoopConf.get());
+      HoodieMetadataFileSystemView fsView = new HoodieMetadataFileSystemView(engineContext, metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), metadataConfig);
+
+      List<String> partitions = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath);
+
+      return partitions.parallelStream().flatMap(p -> fsView.getLatestBaseFiles(p).map(HoodieBaseFile::getFileName));
+    } catch (Exception e) {
+      String msg = "Error checking path :" + basePath;
+      LOG.error(msg, e);
+      throw new HoodieException(msg, e);
+    }
+  }
+
+  private static HoodieMetadataConfig buildMetadataConfig(Configuration conf) {
+    return HoodieMetadataConfig.newBuilder()
+        .enable(conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS))
+        .build();
+  }
+
+  /**
+   * @return Manifest File folder
+   */
+  public String getManifestFolder() {
+    return metaClient.getMetaPath() + Path.SEPARATOR + MANIFEST_FOLDER_NAME;
+  }
+
+  public String getManifestFilePath() {
+    return metaClient.getMetaPath() + Path.SEPARATOR + MANIFEST_FOLDER_NAME +  Path.SEPARATOR + MANIFEST_FILE_NAME;
+  }
+
+  public void setBasePath(String basePath) {
+    this.basePath = basePath;
+  }

Review comment:
       why you want to allow changing basePath? better to make basePath final. once the object created via builder, it should not alter its state to avoid misuse

##########
File path: hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileUtil.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.sync.common.util;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+
+public class ManifestFileUtil {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(ManifestFileUtil.class);
+  private static final String MANIFEST_FOLDER_NAME = "manifest";
+  private static final String MANIFEST_FILE_NAME = "latest-snapshot.csv";
+  private static final String DELIMITER = "\n";
+  private SerializableConfiguration hadoopConf;
+  private String basePath;
+  private transient HoodieLocalEngineContext engineContext;
+  private HoodieTableMetaClient metaClient;
+
+  private ManifestFileUtil(Configuration conf, String basePath) {
+    this.hadoopConf = new SerializableConfiguration(conf);
+    this.basePath = basePath;
+    this.engineContext = new HoodieLocalEngineContext(conf);
+    this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
+  }
+
+  public ManifestFileUtil() {
+  }
+
+  public synchronized void writeManifestFile() {
+    try {
+      Path manifestFilePath = new Path(getManifestFolder(), MANIFEST_FILE_NAME);
+      Option<byte[]> content = Option.of(fetchLatestBaseFilesForAllPartitions().collect(Collectors.joining(DELIMITER)).getBytes());
+      FileIOUtils.createFileInPath(metaClient.getFs(), manifestFilePath, content);
+    } catch (Exception e) {
+      String msg = "Error writing manifest file";
+      LOG.error(msg, e);
+      throw new HoodieException(msg, e);
+    }
+  }
+
+  public Stream<String> fetchLatestBaseFilesForAllPartitions() {
+    try {
+      HoodieMetadataConfig metadataConfig = buildMetadataConfig(hadoopConf.get());
+      HoodieMetadataFileSystemView fsView = new HoodieMetadataFileSystemView(engineContext, metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), metadataConfig);
+
+      List<String> partitions = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath);
+
+      return partitions.parallelStream().flatMap(p -> fsView.getLatestBaseFiles(p).map(HoodieBaseFile::getFileName));
+    } catch (Exception e) {
+      String msg = "Error checking path :" + basePath;
+      LOG.error(msg, e);
+      throw new HoodieException(msg, e);
+    }
+  }
+
+  private static HoodieMetadataConfig buildMetadataConfig(Configuration conf) {
+    return HoodieMetadataConfig.newBuilder()
+        .enable(conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS))
+        .build();
+  }
+
+  /**
+   * @return Manifest File folder
+   */
+  public String getManifestFolder() {
+    return metaClient.getMetaPath() + Path.SEPARATOR + MANIFEST_FOLDER_NAME;
+  }
+
+  public String getManifestFilePath() {
+    return metaClient.getMetaPath() + Path.SEPARATOR + MANIFEST_FOLDER_NAME +  Path.SEPARATOR + MANIFEST_FILE_NAME;
+  }

Review comment:
       can this make use of the other method?

##########
File path: hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ManifestFileUtil.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.sync.common.util;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieLocalEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+
+public class ManifestFileUtil {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(ManifestFileUtil.class);
+  private static final String MANIFEST_FOLDER_NAME = "manifest";
+  private static final String MANIFEST_FILE_NAME = "latest-snapshot.csv";
+  private static final String DELIMITER = "\n";
+  private SerializableConfiguration hadoopConf;
+  private String basePath;
+  private transient HoodieLocalEngineContext engineContext;
+  private HoodieTableMetaClient metaClient;
+
+  private ManifestFileUtil(Configuration conf, String basePath) {
+    this.hadoopConf = new SerializableConfiguration(conf);
+    this.basePath = basePath;
+    this.engineContext = new HoodieLocalEngineContext(conf);
+    this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
+  }
+
+  public ManifestFileUtil() {
+  }
+
+  public synchronized void writeManifestFile() {
+    try {
+      Path manifestFilePath = new Path(getManifestFolder(), MANIFEST_FILE_NAME);
+      Option<byte[]> content = Option.of(fetchLatestBaseFilesForAllPartitions().collect(Collectors.joining(DELIMITER)).getBytes());
+      FileIOUtils.createFileInPath(metaClient.getFs(), manifestFilePath, content);
+    } catch (Exception e) {
+      String msg = "Error writing manifest file";
+      LOG.error(msg, e);
+      throw new HoodieException(msg, e);
+    }
+  }
+
+  public Stream<String> fetchLatestBaseFilesForAllPartitions() {
+    try {
+      HoodieMetadataConfig metadataConfig = buildMetadataConfig(hadoopConf.get());
+      HoodieMetadataFileSystemView fsView = new HoodieMetadataFileSystemView(engineContext, metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), metadataConfig);

Review comment:
       you should probably create a new HoodieMetadataFileSystemView within the lambda below, also create HoodieLocalEngineContext there instead of keep 1 for the entire object. HoodieLocalEngineContext is not serializeable.




-- 
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: commits-unsubscribe@hudi.apache.org

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