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/07/15 03:03:04 UTC

[GitHub] [hudi] microbearz commented on a diff in pull request #5064: [HUDI-3654] Initialize hudi metastore module.

microbearz commented on code in PR #5064:
URL: https://github.com/apache/hudi/pull/5064#discussion_r919693379


##########
hudi-metastore/src/main/java/org/apache/hudi/metastore/client/HoodieMetastoreClient.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.metastore.client;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.metastore.thrift.Table;
+
+import java.util.List;
+
+/**
+ * Hoodie metastore client, is to get/put instants, instant meta, snapshot from/to hoodie metastore.
+ */
+public interface HoodieMetastoreClient {
+
+  Table getTable(String db, String tb);
+
+  void createTable(Table table);
+
+  List<HoodieInstant> listInstants(String db, String tb, int commitNum);
+
+  Option<byte[]> getInstantMeta(String db, String tb, HoodieInstant instant);
+
+  String createNewTimestamp(String db, String tb);
+
+  void createNewInstant(String db, String tb, HoodieInstant instant, Option<byte[]> content);
+
+  void transitionInstantState(String db, String tb, HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> content);
+
+  void deleteInstant(String db, String tb, HoodieInstant instant);
+
+  FileStatus[] listFilesInPartition(String db, String tb, String partition, String timestamp);
+
+  List<String> list_all_partitions(String db, String tb);

Review Comment:
   `list_all_partitions`  why use a different naming rule



##########
hudi-metastore/src/main/java/org/apache/hudi/metastore/HoodieMetastoreServer.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.metastore;
+
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.metastore.service.HoodieMetastoreService;
+import org.apache.hudi.metastore.service.HoodieMetaStoreProxyHandler;
+import org.apache.hudi.metastore.service.PartitionService;
+import org.apache.hudi.metastore.service.SnapshotService;
+import org.apache.hudi.metastore.service.TableService;
+import org.apache.hudi.metastore.service.TimelineService;
+import org.apache.hudi.metastore.store.RelationDBBasedStore;
+import org.apache.hudi.metastore.store.MetadataStore;
+import org.apache.hudi.metastore.thrift.MetaStoreException;
+import org.apache.hudi.metastore.thrift.ThriftHoodieMetastore;
+import org.apache.hudi.metastore.util.TServerSocketWrapper;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.server.TThreadPoolServer;
+import org.apache.thrift.transport.TServerTransport;
+
+import java.lang.reflect.Proxy;
+
+/**
+ * Main class of hoodie metastore.
+ */
+public class HoodieMetastoreServer {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieMetastoreServer.class);
+
+  private static TServer server;
+  private static Thread serverThread;
+  private static MetadataStore metadataStore;
+  private static HoodieMetastoreService metastoreService;
+
+  public static void main(String[] args) {
+    startServer();
+  }
+
+  public static void startServer() {
+    try {
+      if (server != null) {
+        return;
+      }
+      metadataStore = new RelationDBBasedStore();
+      // service
+      TableService tableService = new TableService(metadataStore);
+      PartitionService partitionService = new PartitionService(metadataStore);
+      TimelineService timelineService = new TimelineService(metadataStore);
+      SnapshotService snapshotService = new SnapshotService(metadataStore);
+      HoodieMetastoreService hoodieMetastoreService = new HoodieMetastoreService(tableService,
+          partitionService, timelineService, snapshotService);
+      HoodieMetaStoreProxyHandler proxyHandler = new HoodieMetaStoreProxyHandler(hoodieMetastoreService);
+
+      // start a thrift server
+      ThriftHoodieMetastore.Iface proxy = (ThriftHoodieMetastore.Iface) Proxy
+          .newProxyInstance(HoodieMetaStoreProxyHandler.class.getClassLoader(),
+              new Class[]{ThriftHoodieMetastore.Iface.class}, proxyHandler);
+      ThriftHoodieMetastore.Processor processor = new ThriftHoodieMetastore.Processor(proxy);
+      TServerTransport serverTransport = new TServerSocketWrapper(9090);
+      server = new TThreadPoolServer(new TThreadPoolServer.Args(serverTransport).processor(processor));
+      LOG.info("Starting the server");
+      serverThread = new Thread(() -> server.serve());
+      serverThread.start();
+    } catch (Exception e) {
+      LOG.error("Fail to start the server", e);
+      System.exit(1);
+    }
+  }
+
+  public static ThriftHoodieMetastore.Iface getEmbeddedMetastore() {
+    if (metadataStore == null) {
+      synchronized (HoodieMetastoreServer.class) {
+        if (metadataStore == null) {
+          // TODO: add metastore factory.
+          metadataStore = new RelationDBBasedStore();
+          try {
+            metadataStore.initStore();
+          } catch (MetaStoreException e) {
+            throw new HoodieIOException("Fail to init the embedded metastore," + e.getMessage());

Review Comment:
   better use `throw new HoodieIOException("Fail to init the embedded metastore", e);`
   we shouldn't swallow the origin exception



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