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/06/07 08:57:53 UTC

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

XuQianJin-Stars commented on code in PR #5064:
URL: https://github.com/apache/hudi/pull/5064#discussion_r890956666


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionBasedOnMetastore.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.table.action.commit;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.HoodieMetastoreConfig;
+import org.apache.hudi.common.model.HoodieAvroPayload;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.testutils.HoodieTestTable;
+import org.apache.hudi.common.testutils.RawTripTestPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.testutils.HoodieClientTestBase;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestCopyOnWriteActionBasedOnMetastore extends HoodieClientTestBase {
+  private static final Schema SCHEMA = getSchemaFromResource(TestCopyOnWriteActionBasedOnMetastore.class, "/exampleSchema.avsc");
+
+  @Test
+  public void testInsert() throws Exception {
+    HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withProps(initMetastoreConfig())
+        .withPath(basePath).withSchema(SCHEMA.toString()).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
+            .withRemoteServerPort(timelineServicePort).build()).build();
+    String instantTime = HoodieTestTable.makeNewCommitTime();
+    SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
+    writeClient.startCommitWithTime(instantTime);
+    assertEquals(1, metaClient.reloadActiveTimeline().getInstants().count());
+
+    // Get some records belong to the same partition (2016/01/31)
+    String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
+    String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
+    String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+    String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\","
+        + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}";
+
+    List<HoodieRecord> records = new ArrayList<>();
+    RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
+    records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
+    RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
+    records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
+    RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
+    records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
+    writeClient.insert(jsc.parallelize(records, 1), instantTime);
+    assertEquals(1, metaClient.reloadActiveTimeline().getInstants().count());
+  }
+
+  @Override
+  protected void initMetaClient(HoodieTableType tableType, Properties properties) throws IOException {
+    properties.put(HoodieTableConfig.NAME.key(), "test");

Review Comment:
   Redundant, repeat with `initMetastoreConfig` below.



##########
hudi-metastore/src/main/resources/mybatis/DDLMapper.xml:
##########
@@ -0,0 +1,121 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN"
+        "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+
+<mapper namespace="DDLMapper">
+    <update id="createDBs">
+        CREATE TABLE dbs
+        (
+            db_id BIGINT UNSIGNED PRIMARY KEY AUTO_INCREMENT COMMENT 'uuid',
+            desc VARCHAR(512) COMMENT 'database description',
+            location_uri VARCHAR(512) COMMENT 'database storage path',
+            name VARCHAR(512) UNIQUE COMMENT 'database name',
+            owner_name VARCHAR(512) COMMENT 'database owner' ,
+            owner_type VARCHAR(512) COMMENT 'database owner'

Review Comment:
   All tables here are added with creation time and update time for easy problem analysis



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