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/15 16:39:07 UTC

[GitHub] [hudi] bvaradar commented on a change in pull request #3808: [HUDI-2560] introduce id_based schema to support full schema evolution.

bvaradar commented on a change in pull request #3808:
URL: https://github.com/apache/hudi/pull/3808#discussion_r827175943



##########
File path: hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.internal.schema.io;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.table.timeline.HoodieTimeline.SAVE_SCHEMA_ACTION;
+
+public class FileBasedInternalSchemaStorageManager extends AbstractInternalSchemaStorageManager {
+  private static final Logger LOG = LogManager.getLogger(FileBasedInternalSchemaStorageManager.class);
+
+  public static final String SCHEMA_NAME = ".schema";
+  private final Path baseSchemaPath;
+  private Configuration conf;
+  private HoodieTableMetaClient metaClient;
+
+  public FileBasedInternalSchemaStorageManager(Configuration conf, Path baseTablePath) {
+    Path metaPath = new Path(baseTablePath, ".hoodie");
+    this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
+    this.conf = conf;
+    this.metaClient = HoodieTableMetaClient.builder().setBasePath(metaPath.getParent().toString()).setConf(conf).build();
+  }
+
+  public FileBasedInternalSchemaStorageManager(HoodieTableMetaClient metaClient) {
+    Path metaPath = new Path(metaClient.getBasePath(), ".hoodie");
+    this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
+    this.conf = metaClient.getHadoopConf();
+    this.metaClient = metaClient;
+  }
+
+  @Override
+  public void persistHistorySchemaStr(String instantTime, String historySchemaStr) {
+    cleanResidualFiles();
+    HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+    HoodieInstant hoodieInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, SAVE_SCHEMA_ACTION, instantTime);
+    timeline.createNewInstant(hoodieInstant);
+    byte[] writeContent = historySchemaStr.getBytes(StandardCharsets.UTF_8);
+    timeline.transitionRequestedToInflight(hoodieInstant, Option.empty());
+    timeline.saveAsComplete(new HoodieInstant(HoodieInstant.State.INFLIGHT, hoodieInstant.getAction(), hoodieInstant.getTimestamp()), Option.of(writeContent));
+    LOG.info(String.format("persist history schema success on commit time: %s", instantTime));
+  }
+
+  private void cleanResidualFiles() {
+    List<String> validateCommits = getValidInstants();

Review comment:
       Rename validateCommits to validInstants

##########
File path: hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.internal.schema;
+
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.internal.schema.Types.Field;
+import org.apache.hudi.internal.schema.Types.RecordType;
+import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Internal schema for hudi table.
+ * used to support schema evolution.
+ */
+public class InternalSchema implements Serializable {
+
+  private static final long DEFAULT_VERSION_ID = 0;
+
+  private final RecordType record;
+
+  private int maxColumnId;
+  private long versionId;
+
+  private transient Map<Integer, Field> idToField = null;
+  private transient Map<String, Integer> nameToId = null;
+  private transient Map<Integer, String> idToName = null;
+
+  public InternalSchema(List<Field> columns) {
+    this(DEFAULT_VERSION_ID, columns);
+  }
+
+  public InternalSchema(Field... columns) {
+    this(DEFAULT_VERSION_ID, Arrays.asList(columns));
+  }
+
+  public InternalSchema(long versionId, List<Field> cols) {
+    this.versionId = versionId;
+    this.record = RecordType.get(cols);
+    buildIdToName();
+    maxColumnId = idToName.keySet().stream().max(Comparator.comparing(Integer::valueOf)).get();
+  }
+
+  public InternalSchema(long versionId, int maxColumnId, List<Field> cols) {
+    this.maxColumnId = maxColumnId;
+    this.versionId = versionId;
+    this.record = RecordType.get(cols);
+    buildIdToName();
+  }
+
+  public InternalSchema(long versionId, int maxColumnId, Field... cols) {
+    this(versionId, maxColumnId, Arrays.asList(cols));
+  }
+
+  public RecordType getRecord() {
+    return record;
+  }
+
+  private Map<Integer, String> buildIdToName() {
+    if (idToName == null) {
+      idToName = InternalSchemaUtils.buildIdToName(record);
+    }
+    return idToName;
+  }
+
+  private Map<String, Integer> buildNameToId() {
+    if (nameToId == null) {
+      if (idToName != null && !idToName.isEmpty()) {
+        nameToId = idToName.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
+        return nameToId;
+      }
+      nameToId = InternalSchemaUtils.buildNameToId(record);
+    }
+    return nameToId;
+  }
+
+  private Map<Integer, Field> buildIdToField() {
+    if (idToField == null) {
+      idToField = InternalSchemaUtils.buildIdToField(record);
+    }
+    return idToField;
+  }
+
+  /**
+   * get all columns full name.
+   */
+  public List<String> getAllColsFullName() {
+    if (nameToId == null) {
+      nameToId = InternalSchemaUtils.buildNameToId(record);
+    }
+    return Arrays.asList(nameToId.keySet().toArray(new String[0]));
+  }
+
+  /**
+   * set the version ID for this schema.
+   */
+  public InternalSchema setSchemaId(long versionId) {
+    this.versionId = versionId;
+    return this;
+  }
+
+  /**
+   * Returns the version ID for this schema.
+   */
+  public long schemaId() {
+    return this.versionId;
+  }
+
+  /**
+   * set the version ID for this schema.
+   */
+  public void setMax_column_id(int maxColumnId) {

Review comment:
       Please make all the method names camelCase. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/internal/schema/io/FileBasedInternalSchemaStorageManager.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.internal.schema.io;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.table.timeline.HoodieTimeline.SAVE_SCHEMA_ACTION;
+
+public class FileBasedInternalSchemaStorageManager extends AbstractInternalSchemaStorageManager {
+  private static final Logger LOG = LogManager.getLogger(FileBasedInternalSchemaStorageManager.class);
+
+  public static final String SCHEMA_NAME = ".schema";
+  private final Path baseSchemaPath;
+  private Configuration conf;
+  private HoodieTableMetaClient metaClient;
+
+  public FileBasedInternalSchemaStorageManager(Configuration conf, Path baseTablePath) {
+    Path metaPath = new Path(baseTablePath, ".hoodie");
+    this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
+    this.conf = conf;
+    this.metaClient = HoodieTableMetaClient.builder().setBasePath(metaPath.getParent().toString()).setConf(conf).build();
+  }
+
+  public FileBasedInternalSchemaStorageManager(HoodieTableMetaClient metaClient) {
+    Path metaPath = new Path(metaClient.getBasePath(), ".hoodie");
+    this.baseSchemaPath = new Path(metaPath, SCHEMA_NAME);
+    this.conf = metaClient.getHadoopConf();
+    this.metaClient = metaClient;
+  }
+
+  @Override
+  public void persistHistorySchemaStr(String instantTime, String historySchemaStr) {
+    cleanResidualFiles();
+    HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+    HoodieInstant hoodieInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, SAVE_SCHEMA_ACTION, instantTime);
+    timeline.createNewInstant(hoodieInstant);
+    byte[] writeContent = historySchemaStr.getBytes(StandardCharsets.UTF_8);
+    timeline.transitionRequestedToInflight(hoodieInstant, Option.empty());
+    timeline.saveAsComplete(new HoodieInstant(HoodieInstant.State.INFLIGHT, hoodieInstant.getAction(), hoodieInstant.getTimestamp()), Option.of(writeContent));
+    LOG.info(String.format("persist history schema success on commit time: %s", instantTime));
+  }
+
+  private void cleanResidualFiles() {
+    List<String> validateCommits = getValidInstants();
+    try {
+      FileSystem fs = baseSchemaPath.getFileSystem(conf);
+      if (fs.exists(baseSchemaPath)) {
+        List<String> candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile())
+            .map(file -> file.getPath().getName()).collect(Collectors.toList());
+        List<String> residualSchemaFiles = candidateSchemaFiles.stream().filter(f -> !validateCommits.contains(f.split("\\.")[0])).collect(Collectors.toList());
+        // clean residual files
+        residualSchemaFiles.forEach(f -> {
+          try {
+            fs.delete(new Path(metaClient.getSchemaFolderName(), f));
+          } catch (IOException o) {
+            throw new HoodieException(o);
+          }
+        });
+      }
+    } catch (IOException e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  public void cleanOldFiles(List<String> validateCommits) {
+    try {
+      FileSystem fs = baseSchemaPath.getFileSystem(conf);
+      if (fs.exists(baseSchemaPath)) {
+        List<String> candidateSchemaFiles = Arrays.stream(fs.listStatus(baseSchemaPath)).filter(f -> f.isFile())
+            .map(file -> file.getPath().getName()).collect(Collectors.toList());
+        List<String> validateSchemaFiles = candidateSchemaFiles.stream().filter(f -> validateCommits.contains(f.split("\\.")[0])).collect(Collectors.toList());

Review comment:
       nit: validateSchemaFiles => validSchemaFiles

##########
File path: hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.internal.schema.action;
+
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.Type;
+import org.apache.hudi.internal.schema.Types;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * auxiliary class.
+ * help to merge file schema and query schema to produce final read schema for avro/parquet file
+ */
+public class InternalSchemaMerger {
+  private final InternalSchema fileSchema;
+  private final InternalSchema querySchema;
+  // now there exist some bugs when we use spark update/merge api,

Review comment:
       Can you add some pointers to the issue  GH/Jira issue. 




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