partitionsToAdd) {
if (partitionsToAdd.isEmpty()) {
@@ -429,6 +480,120 @@ public void createTable(String tableName,
}
}
+ /**
+ * This will manage partitions indexes. Users can activate/deactivate them on existing tables.
+ * Removing index definition, will result in dropping the index.
+ *
+ * reference doc for partition indexes:
+ * https://docs.aws.amazon.com/glue/latest/dg/partition-indexes.html#partition-index-getpartitions
+ *
+ * @param tableName
+ */
+ public void managePartitionIndexes(String tableName) throws ExecutionException, InterruptedException {
+ if (!config.getBooleanOrDefault(META_SYNC_PARTITION_INDEX_FIELDS_ENABLE)) {
+ // deactivate indexing if enabled
+ if (getPartitionIndexEnable(tableName)) {
+ LOG.warn("Deactivating partition indexing");
+ updatePartitionIndexEnable(tableName, false);
+ }
+ // also drop all existing indexes
+ GetPartitionIndexesRequest indexesRequest = GetPartitionIndexesRequest.builder().databaseName(databaseName).tableName(tableName).build();
+ GetPartitionIndexesResponse existingIdxsResp = awsGlue.getPartitionIndexes(indexesRequest).get();
+ for (PartitionIndexDescriptor idsToDelete : existingIdxsResp.partitionIndexDescriptorList()) {
+ LOG.warn("Dropping partition index: " + idsToDelete.indexName());
+ DeletePartitionIndexRequest idxToDelete = DeletePartitionIndexRequest.builder()
+ .databaseName(databaseName).tableName(tableName).indexName(idsToDelete.indexName()).build();
+ awsGlue.deletePartitionIndex(idxToDelete).get();
+ }
+ } else {
+ // activate indexing usage if disabled
+ if (!getPartitionIndexEnable(tableName)) {
+ LOG.warn("Activating partition indexing");
+ updatePartitionIndexEnable(tableName, true);
+ }
+
+ // get indexes to be created
+ List> partitionsIndexNeeded = parsePartitionsIndexConfig();
+ // get existing indexes
+ GetPartitionIndexesRequest indexesRequest = GetPartitionIndexesRequest.builder()
+ .databaseName(databaseName).tableName(tableName).build();
+ GetPartitionIndexesResponse existingIdxsResp = awsGlue.getPartitionIndexes(indexesRequest).get();
+
+ // for each existing index remove if not relevant anymore
+ boolean indexesChanges = false;
+ for (PartitionIndexDescriptor existingIdx: existingIdxsResp.partitionIndexDescriptorList()) {
+ List idxColumns = existingIdx.keys().stream().map(key -> key.name()).collect(Collectors.toList());
+ Boolean toBeRemoved = true;
+ for (List neededIdx : partitionsIndexNeeded) {
+ if (neededIdx.equals(idxColumns)) {
+ toBeRemoved = false;
+ }
+ }
+ if (toBeRemoved) {
+ indexesChanges = true;
+ DeletePartitionIndexRequest idxToDelete = DeletePartitionIndexRequest.builder()
+ .databaseName(databaseName).tableName(tableName).indexName(existingIdx.indexName()).build();
+ LOG.warn("Dropping irrelevant index: " + existingIdx.indexName());
+ awsGlue.deletePartitionIndex(idxToDelete).get();
+ }
+ }
+ if (indexesChanges) { // refresh indexes list
+ existingIdxsResp = awsGlue.getPartitionIndexes(indexesRequest).get();
+ }
+
+ // for each needed index create if not exist
+ for (List neededIdx : partitionsIndexNeeded) {
+ Boolean toBeCreated = true;
+ for (PartitionIndexDescriptor existingIdx: existingIdxsResp.partitionIndexDescriptorList()) {
+ List collect = existingIdx.keys().stream().map(key -> key.name()).collect(Collectors.toList());
+ if (collect.equals(neededIdx)) {
+ toBeCreated = false;
+ }
+ }
+ if (toBeCreated) {
+ String newIdxName = String.format("hudi_managed_%s", neededIdx.toString());
+ PartitionIndex newIdx = PartitionIndex.builder()
+ .indexName(newIdxName)
+ .keys(neededIdx).build();
+ LOG.warn("Creating new partition index: " + newIdxName);
+ CreatePartitionIndexRequest creationRequest = CreatePartitionIndexRequest.builder()
+ .databaseName(databaseName).tableName(tableName).partitionIndex(newIdx).build();
+ awsGlue.createPartitionIndex(creationRequest).get();
+ }
+ }
+ }
+ }
+
+ protected List> parsePartitionsIndexConfig() {
+ config.setDefaultValue(META_SYNC_PARTITION_INDEX_FIELDS);
+ String rawPartitionIndex = config.getString(META_SYNC_PARTITION_INDEX_FIELDS);
+ List> indexes = Arrays.stream(rawPartitionIndex.split(","))
+ .map(idx -> Arrays.stream(idx.split(";"))
+ .collect(Collectors.toList())).collect(Collectors.toList());
+ if (indexes.size() > PARTITION_INDEX_MAX_NUMBER) {
+ LOG.warn(String.format("Only considering first %s indexes", PARTITION_INDEX_MAX_NUMBER));
+ return indexes.subList(0, PARTITION_INDEX_MAX_NUMBER);
+ }
+ return indexes;
+ }
+
+ public Boolean getPartitionIndexEnable(String tableName) {
+ try {
+ Table table = getTable(awsGlue, databaseName, tableName);
+ return Boolean.valueOf(table.parameters().get(GLUE_PARTITION_INDEX_ENABLE));
+ } catch (Exception e) {
+ throw new HoodieGlueSyncException("Fail to get parameter " + GLUE_PARTITION_INDEX_ENABLE + " time for " + tableId(databaseName, tableName), e);
+ }
+ }
+
+ public void updatePartitionIndexEnable(String tableName, Boolean enable) {
+ try {
+ updateTableParameters(awsGlue, databaseName, tableName, Collections.singletonMap(GLUE_PARTITION_INDEX_ENABLE, enable.toString()), false);
+ } catch (Exception e) {
+ throw new HoodieGlueSyncException("Fail to update parameter " + GLUE_PARTITION_INDEX_ENABLE + " time for " + tableId(databaseName, tableName), e);
+ }
+ }
+
@Override
public Map getMetastoreSchema(String tableName) {
try {
@@ -450,6 +615,14 @@ public Map getMetastoreSchema(String tableName) {
}
}
+ @Override
+ public List getMetastoreFieldSchemas(String tableName) {
+ Map schema = getMetastoreSchema(tableName);
+ return schema.entrySet().stream()
+ .map(f -> new FieldSchema(f.getKey(), f.getValue()))
+ .collect(Collectors.toList());
+ }
+
@Override
public boolean tableExists(String tableName) {
GetTableRequest request = GetTableRequest.builder()
@@ -537,6 +710,16 @@ public void updateLastCommitTimeSynced(String tableName) {
} catch (Exception e) {
throw new HoodieGlueSyncException("Fail to update last sync commit time for " + tableId(databaseName, tableName), e);
}
+ try {
+ // as a side effect, we also refresh the partition indexes if needed
+ // people may wan't to add indexes, without re-creating the table
+ // therefore we call this at each commit as a workaround
+ managePartitionIndexes(tableName);
+ } catch (ExecutionException e) {
+ LOG.warn("An indexation process is currently running.", e);
+ } catch (Exception e) {
+ LOG.warn("Something went wrong with partition index", e);
+ }
}
@Override
@@ -554,6 +737,11 @@ public void deleteLastReplicatedTimeStamp(String tableName) {
throw new UnsupportedOperationException("Not supported: `deleteLastReplicatedTimeStamp`");
}
+ @Override
+ public String generatePushDownFilter(List writtenPartitions, List partitionFields) {
+ return new GluePartitionFilterGenerator().generatePushDownFilter(writtenPartitions, partitionFields, (HiveSyncConfig) config);
+ }
+
private List getColumnsFromSchema(Map mapSchema) {
List cols = new ArrayList<>();
for (String key : mapSchema.keySet()) {
diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java
index eed9486d69cd..e86a6b99f5cc 100644
--- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java
+++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java
@@ -19,7 +19,7 @@
package org.apache.hudi.aws.sync;
import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HiveSyncTool;
@@ -62,7 +62,7 @@ public static void main(String[] args) {
}
// HiveConf needs to load fs conf to allow instantiation via AWSGlueClientFactory
TypedProperties props = params.toProps();
- Configuration hadoopConf = FSUtils.getFs(props.getString(META_SYNC_BASE_PATH.key()), new Configuration()).getConf();
+ Configuration hadoopConf = HadoopFSUtils.getFs(props.getString(META_SYNC_BASE_PATH.key()), new Configuration()).getConf();
try (AwsGlueCatalogSyncTool tool = new AwsGlueCatalogSyncTool(props, hadoopConf)) {
tool.syncHoodieTable();
}
diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/util/GlueFilterGenVisitor.java
similarity index 63%
rename from hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java
rename to hudi-aws/src/main/java/org/apache/hudi/aws/sync/util/GlueFilterGenVisitor.java
index 51c53f368fb9..859e01032103 100644
--- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java
+++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/util/GlueFilterGenVisitor.java
@@ -7,7 +7,7 @@
* "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
+ * 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,
@@ -16,20 +16,17 @@
* limitations under the License.
*/
-package org.apache.hudi.adapter;
+package org.apache.hudi.aws.sync.util;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.hudi.hive.util.FilterGenVisitor;
+
+public class GlueFilterGenVisitor extends FilterGenVisitor {
-/**
- * Adapter clazz for {@code AbstractStreamOperator}.
- */
-public abstract class AbstractStreamOperatorAdapter extends AbstractStreamOperator {
@Override
- public void close() throws Exception {
- super.dispose();
+ protected String quoteStringLiteral(String value) {
+ // Glue uses jSQLParser.
+ // https://jsqlparser.github.io/JSqlParser/usage.html#define-the-parser-features
+ return "'" + (value.contains("'") ? value.replaceAll("'", "''") : value) + "'";
}
- public void finish() throws Exception {
- super.close();
- }
}
diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/RowColumnVector.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/util/GluePartitionFilterGenerator.java
similarity index 66%
rename from hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/RowColumnVector.java
rename to hudi-aws/src/main/java/org/apache/hudi/aws/sync/util/GluePartitionFilterGenerator.java
index bd0e9bbe7de7..c9a8605a2270 100644
--- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/data/vector/RowColumnVector.java
+++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/util/GluePartitionFilterGenerator.java
@@ -7,7 +7,7 @@
* "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
+ * 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,
@@ -16,15 +16,14 @@
* limitations under the License.
*/
-package org.apache.hudi.table.data.vector;
+package org.apache.hudi.aws.sync.util;
-import org.apache.hudi.table.data.ColumnarRowData;
+import org.apache.hudi.expression.Expression;
+import org.apache.hudi.hive.util.PartitionFilterGenerator;
-import org.apache.flink.table.data.vector.ColumnVector;
+public class GluePartitionFilterGenerator extends PartitionFilterGenerator {
-/**
- * Row column vector.
- */
-public interface RowColumnVector extends ColumnVector {
- ColumnarRowData getRow(int i);
+ protected String generateFilterString(Expression filter) {
+ return filter.accept(new GlueFilterGenVisitor());
+ }
}
diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/GlueCatalogSyncClientConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/GlueCatalogSyncClientConfig.java
index efffae5bd893..21244e651547 100644
--- a/hudi-aws/src/main/java/org/apache/hudi/config/GlueCatalogSyncClientConfig.java
+++ b/hudi-aws/src/main/java/org/apache/hudi/config/GlueCatalogSyncClientConfig.java
@@ -22,6 +22,9 @@
import org.apache.hudi.common.config.ConfigGroups;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
/**
* Hoodie Configs for Glue.
@@ -46,4 +49,20 @@ public class GlueCatalogSyncClientConfig extends HoodieConfig {
.markAdvanced()
.sinceVersion("0.14.0")
.withDocumentation("Makes athena use the metadata table to list partitions and files. Currently it won't benefit from other features such stats indexes");
+
+ public static final ConfigProperty META_SYNC_PARTITION_INDEX_FIELDS_ENABLE = ConfigProperty
+ .key(GLUE_CLIENT_PROPERTY_PREFIX + "partition_index_fields.enable")
+ .defaultValue(false)
+ .sinceVersion("1.0.0")
+ .withDocumentation("Enable aws glue partition index feature, to speedup partition based query pattern");
+
+ public static final ConfigProperty META_SYNC_PARTITION_INDEX_FIELDS = ConfigProperty
+ .key(GLUE_CLIENT_PROPERTY_PREFIX + "partition_index_fields")
+ .noDefaultValue()
+ .withInferFunction(cfg -> Option.ofNullable(cfg.getString(HoodieTableConfig.PARTITION_FIELDS))
+ .or(() -> Option.ofNullable(cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME))))
+ .sinceVersion("1.0.0")
+ .withDocumentation(String.join(" ", "Specify the partitions fields to index on aws glue. Separate the fields by semicolon.",
+ "By default, when the feature is enabled, all the partition will be indexed.",
+ "You can create up to three indexes, separate them by comma. Eg: col1;col2;col3,col2,col3"));
}
diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java
index 45d6878fa3df..8eb76573d0e1 100644
--- a/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java
+++ b/hudi-aws/src/main/java/org/apache/hudi/config/HoodieAWSConfig.java
@@ -69,6 +69,20 @@ public class HoodieAWSConfig extends HoodieConfig {
.sinceVersion("0.10.0")
.withDocumentation("AWS session token");
+ public static final ConfigProperty AWS_GLUE_ENDPOINT = ConfigProperty
+ .key("hoodie.aws.glue.endpoint")
+ .noDefaultValue()
+ .markAdvanced()
+ .sinceVersion("0.14.2")
+ .withDocumentation("Aws glue endpoint");
+
+ public static final ConfigProperty AWS_GLUE_REGION = ConfigProperty
+ .key("hoodie.aws.glue.region")
+ .noDefaultValue()
+ .markAdvanced()
+ .sinceVersion("0.14.2")
+ .withDocumentation("Aws glue endpoint");
+
private HoodieAWSConfig() {
super();
}
diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/sync/ITTestGluePartitionPushdown.java b/hudi-aws/src/test/java/org/apache/hudi/aws/sync/ITTestGluePartitionPushdown.java
new file mode 100644
index 000000000000..b0aa34bdfce1
--- /dev/null
+++ b/hudi-aws/src/test/java/org/apache/hudi/aws/sync/ITTestGluePartitionPushdown.java
@@ -0,0 +1,159 @@
+/*
+ * 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.aws.sync;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieAvroPayload;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.config.HoodieAWSConfig;
+import org.apache.hudi.hive.HiveSyncConfig;
+import org.apache.hudi.sync.common.model.FieldSchema;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.glue.model.Column;
+import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.CreatePartitionRequest;
+import software.amazon.awssdk.services.glue.model.CreateTableRequest;
+import software.amazon.awssdk.services.glue.model.DatabaseInput;
+import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.DeleteTableRequest;
+import software.amazon.awssdk.services.glue.model.PartitionInput;
+import software.amazon.awssdk.services.glue.model.SerDeInfo;
+import software.amazon.awssdk.services.glue.model.StorageDescriptor;
+import software.amazon.awssdk.services.glue.model.TableInput;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hudi.hive.HiveSyncConfig.HIVE_SYNC_FILTER_PUSHDOWN_MAX_SIZE;
+import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH;
+import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
+
+public class ITTestGluePartitionPushdown {
+
+ private static final String MOTO_ENDPOINT = "http://localhost:5000";
+ private static final String DB_NAME = "db_name";
+ private static final String TABLE_NAME = "tbl_name";
+ private String basePath = Files.createTempDirectory("hivesynctest" + Instant.now().toEpochMilli()).toUri().toString();
+ private String tablePath = basePath + "/" + TABLE_NAME;
+ private TypedProperties hiveSyncProps;
+ private AWSGlueCatalogSyncClient glueSync;
+ private FileSystem fileSystem;
+ private Column[] partitionsColumn = {Column.builder().name("part1").type("int").build(), Column.builder().name("part2").type("string").build()};
+ List partitionsFieldSchema = Arrays.asList(new FieldSchema("part1", "int"), new FieldSchema("part2", "string"));
+
+ public ITTestGluePartitionPushdown() throws IOException {}
+
+ @BeforeEach
+ public void setUp() throws Exception {
+ hiveSyncProps = new TypedProperties();
+ hiveSyncProps.setProperty(HoodieAWSConfig.AWS_ACCESS_KEY.key(), "dummy");
+ hiveSyncProps.setProperty(HoodieAWSConfig.AWS_SECRET_KEY.key(), "dummy");
+ hiveSyncProps.setProperty(HoodieAWSConfig.AWS_SESSION_TOKEN.key(), "dummy");
+ hiveSyncProps.setProperty(HoodieAWSConfig.AWS_GLUE_ENDPOINT.key(), MOTO_ENDPOINT);
+ hiveSyncProps.setProperty(HoodieAWSConfig.AWS_GLUE_REGION.key(), "eu-west-1");
+ hiveSyncProps.setProperty(META_SYNC_BASE_PATH.key(), tablePath);
+ hiveSyncProps.setProperty(META_SYNC_DATABASE_NAME.key(), DB_NAME);
+
+ HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(hiveSyncProps, new Configuration());
+ fileSystem = hiveSyncConfig.getHadoopFileSystem();
+ fileSystem.mkdirs(new Path(tablePath));
+ Configuration configuration = new Configuration();
+ HoodieTableMetaClient.withPropertyBuilder()
+ .setTableType(HoodieTableType.COPY_ON_WRITE)
+ .setTableName(TABLE_NAME)
+ .setPayloadClass(HoodieAvroPayload.class)
+ .initTable(configuration, tablePath);
+
+ glueSync = new AWSGlueCatalogSyncClient(new HiveSyncConfig(hiveSyncProps));
+ glueSync.awsGlue.createDatabase(CreateDatabaseRequest.builder().databaseInput(DatabaseInput.builder().name(DB_NAME).build()).build()).get();
+
+ glueSync.awsGlue.createTable(CreateTableRequest.builder().databaseName(DB_NAME)
+ .tableInput(TableInput.builder().name(TABLE_NAME).partitionKeys(
+ partitionsColumn)
+ .storageDescriptor(
+ StorageDescriptor.builder()
+ .serdeInfo(SerDeInfo.builder().serializationLibrary("").build())
+ .location(tablePath)
+ .columns(
+ Column.builder().name("col1").type("string").build()
+ )
+ .build())
+ .build()).build()).get();
+ }
+
+ @AfterEach
+ public void teardown() throws Exception {
+ glueSync.awsGlue.deleteTable(DeleteTableRequest.builder().databaseName(DB_NAME).name(TABLE_NAME).build()).get();
+ glueSync.awsGlue.deleteDatabase(DeleteDatabaseRequest.builder().name(DB_NAME).build()).get();
+ fileSystem.delete(new Path(tablePath), true);
+ }
+
+ private void createPartitions(String...partitions) throws ExecutionException, InterruptedException {
+ glueSync.awsGlue.createPartition(CreatePartitionRequest.builder().databaseName(DB_NAME).tableName(TABLE_NAME)
+ .partitionInput(PartitionInput.builder()
+ .storageDescriptor(StorageDescriptor.builder().columns(partitionsColumn).build())
+ .values(partitions).build()).build()).get();
+ }
+
+ @Test
+ public void testEmptyPartitionShouldReturnEmpty() {
+ Assertions.assertEquals(0, glueSync.getPartitionsByFilter(TABLE_NAME,
+ glueSync.generatePushDownFilter(Arrays.asList("1/bar"), partitionsFieldSchema)).size());
+ }
+
+ @Test
+ public void testPresentPartitionShouldReturnIt() throws ExecutionException, InterruptedException {
+ createPartitions("1", "b'ar");
+ Assertions.assertEquals(1, glueSync.getPartitionsByFilter(TABLE_NAME,
+ glueSync.generatePushDownFilter(Arrays.asList("1/b'ar", "2/foo", "1/b''ar"), partitionsFieldSchema)).size());
+ }
+
+ @Test
+ public void testPresentPartitionShouldReturnAllWhenExpressionFilterLengthTooLong() throws ExecutionException, InterruptedException {
+ createPartitions("1", "b'ar");
+
+ // this will generate an expression larger than GLUE_EXPRESSION_MAX_CHARS
+ List tooLargePartitionPredicate = new ArrayList<>();
+ for (int i = 0; i < 500; i++) {
+ tooLargePartitionPredicate.add(i + "/foo");
+ }
+ Assertions.assertEquals(1, glueSync.getPartitionsByFilter(TABLE_NAME,
+ glueSync.generatePushDownFilter(tooLargePartitionPredicate, partitionsFieldSchema)).size(),
+ "Should fallback to listing all existing partitions");
+
+ // now set the pushdown max size to a low value to transform the expression in lower/upper bound
+ hiveSyncProps.setProperty(HIVE_SYNC_FILTER_PUSHDOWN_MAX_SIZE.key(), "10");
+ glueSync = new AWSGlueCatalogSyncClient(new HiveSyncConfig(hiveSyncProps));
+ Assertions.assertEquals(0, glueSync.getPartitionsByFilter(TABLE_NAME,
+ glueSync.generatePushDownFilter(tooLargePartitionPredicate, partitionsFieldSchema)).size(),
+ "No partitions should match");
+ }
+}
diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/sync/TestGluePartitionPushdown.java b/hudi-aws/src/test/java/org/apache/hudi/aws/sync/TestGluePartitionPushdown.java
new file mode 100644
index 000000000000..d0fe7bf2922d
--- /dev/null
+++ b/hudi-aws/src/test/java/org/apache/hudi/aws/sync/TestGluePartitionPushdown.java
@@ -0,0 +1,143 @@
+/*
+ * 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.aws.sync;
+
+import org.apache.hudi.aws.sync.util.GluePartitionFilterGenerator;
+import org.apache.hudi.hive.HiveSyncConfig;
+import org.apache.hudi.sync.common.model.FieldSchema;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.hudi.hive.HiveSyncConfig.HIVE_SYNC_FILTER_PUSHDOWN_MAX_SIZE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestGluePartitionPushdown {
+
+ GluePartitionFilterGenerator partitionFilterGenerator = new GluePartitionFilterGenerator();
+ @Test
+ public void testPushDownFilters() {
+ Properties props = new Properties();
+ HiveSyncConfig config = new HiveSyncConfig(props);
+ List partitionFieldSchemas = new ArrayList<>(4);
+ partitionFieldSchemas.add(new FieldSchema("date", "date"));
+ partitionFieldSchemas.add(new FieldSchema("year", "string"));
+ partitionFieldSchemas.add(new FieldSchema("month", "int"));
+ partitionFieldSchemas.add(new FieldSchema("day", "bigint"));
+
+ List writtenPartitions = new ArrayList<>();
+ writtenPartitions.add("2022-09-01/2022/9/1");
+ assertEquals("(((date = 2022-09-01 AND year = '2022') AND month = 9) AND day = 1)",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+
+ writtenPartitions.add("2022-09-02/2022/9/2");
+ assertEquals(
+ "((((date = 2022-09-01 AND year = '2022') AND month = 9) AND day = 1) OR (((date = 2022-09-02 AND year = '2022') AND month = 9) AND day = 2))",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+
+ // If there are incompatible types to convert as filters inside partition
+ partitionFieldSchemas.clear();
+ writtenPartitions.clear();
+ partitionFieldSchemas.add(new FieldSchema("date", "date"));
+ partitionFieldSchemas.add(new FieldSchema("finished", "boolean"));
+
+ writtenPartitions.add("2022-09-01/true");
+ assertEquals("date = 2022-09-01",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+ writtenPartitions.add("2022-09-02/true");
+ assertEquals("(date = 2022-09-01 OR date = 2022-09-02)",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+
+ // If no compatible types matched to convert as filters
+ partitionFieldSchemas.clear();
+ writtenPartitions.clear();
+ partitionFieldSchemas.add(new FieldSchema("finished", "boolean"));
+
+ writtenPartitions.add("true");
+ assertEquals("",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+ writtenPartitions.add("false");
+ assertEquals("",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+
+ // If no compatible types matched to convert as filters
+ partitionFieldSchemas.clear();
+ writtenPartitions.clear();
+ partitionFieldSchemas.add(new FieldSchema("status", "string"));
+ writtenPartitions.add("to_be_'escaped");
+ assertEquals("status = 'to_be_''escaped'",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+ }
+
+ @Test
+ public void testPushDownFilterIfExceedLimit() {
+ Properties props = new Properties();
+ props.put(HIVE_SYNC_FILTER_PUSHDOWN_MAX_SIZE.key(), "0");
+ HiveSyncConfig config = new HiveSyncConfig(props);
+ List partitionFieldSchemas = new ArrayList<>(4);
+ partitionFieldSchemas.add(new FieldSchema("date", "date"));
+ partitionFieldSchemas.add(new FieldSchema("year", "string"));
+ partitionFieldSchemas.add(new FieldSchema("month", "int"));
+ partitionFieldSchemas.add(new FieldSchema("day", "bigint"));
+
+ List writtenPartitions = new ArrayList<>();
+ writtenPartitions.add("2022-09-01/2022/9/1");
+
+ assertEquals("(((date = 2022-09-01 AND year = '2022') AND month = 9) AND day = 1)",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+
+ writtenPartitions.add("2022-09-02/2022/9/2");
+ writtenPartitions.add("2022-09-03/2022/9/2");
+ writtenPartitions.add("2022-09-04/2022/9/2");
+ assertEquals(
+ "((((date >= 2022-09-01 AND date <= 2022-09-04) AND year = '2022') AND month = 9) AND (day >= 1 AND day <= 2))",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+
+ // If there are incompatible types to convert as filters inside partition
+ partitionFieldSchemas.clear();
+ writtenPartitions.clear();
+ partitionFieldSchemas.add(new FieldSchema("date", "date"));
+ partitionFieldSchemas.add(new FieldSchema("finished", "boolean"));
+
+ writtenPartitions.add("2022-09-01/true");
+ assertEquals("date = 2022-09-01",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+ writtenPartitions.add("2022-09-02/true");
+ writtenPartitions.add("2022-09-03/false");
+ writtenPartitions.add("2022-09-04/false");
+ assertEquals("(date >= 2022-09-01 AND date <= 2022-09-04)",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+
+ // If no compatible types matched to convert as filters
+ partitionFieldSchemas.clear();
+ writtenPartitions.clear();
+ partitionFieldSchemas.add(new FieldSchema("finished", "boolean"));
+
+ writtenPartitions.add("true");
+ assertEquals("",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+ writtenPartitions.add("false");
+ writtenPartitions.add("false");
+ writtenPartitions.add("false");
+ assertEquals("",
+ partitionFilterGenerator.generatePushDownFilter(writtenPartitions, partitionFieldSchemas, config));
+ }
+}
diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml
index 205e523315bc..8a6875a9df46 100644
--- a/hudi-cli/pom.xml
+++ b/hudi-cli/pom.xml
@@ -19,7 +19,7 @@
hudi
org.apache.hudi
- 0.14.1
+ 0.15.0-SNAPSHOT
4.0.0
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java
index 7b54760cddce..7cec0172b157 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java
@@ -21,10 +21,10 @@
import org.apache.hudi.cli.utils.SparkTempViewProvider;
import org.apache.hudi.cli.utils.TempViewProvider;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -72,7 +72,7 @@ private static void setLayoutVersion(Integer layoutVersion) {
public static boolean initConf() {
if (HoodieCLI.conf == null) {
- HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration());
+ HoodieCLI.conf = HadoopFSUtils.prepareHadoopConf(new Configuration());
return true;
}
return false;
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
index 90724929df40..075a57d541c0 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
@@ -26,7 +26,6 @@
import org.apache.hudi.cli.commands.SparkMain.SparkCommand;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
@@ -38,6 +37,7 @@
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -110,11 +110,11 @@ public String showArchivedCommits(
if (folder != null && !folder.isEmpty()) {
archivePath = new Path(basePath + "/.hoodie/" + folder);
}
- FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
+ FileStatus[] fsStatuses = HadoopFSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
List allStats = new ArrayList<>();
for (FileStatus fs : fsStatuses) {
// read the archived file
- Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
+ Reader reader = HoodieLogFormat.newReader(HadoopFSUtils.getFs(basePath, HoodieCLI.conf),
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
List readRecords = new ArrayList<>();
@@ -184,11 +184,11 @@ public String showCommits(
String basePath = metaClient.getBasePath();
Path archivePath = new Path(metaClient.getArchivePath() + "/.commits_.archive*");
FileStatus[] fsStatuses =
- FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
+ HadoopFSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
List allCommits = new ArrayList<>();
for (FileStatus fs : fsStatuses) {
// read the archived file
- HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
+ HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(HadoopFSUtils.getFs(basePath, HoodieCLI.conf),
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
List readRecords = new ArrayList<>();
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java
index c9cebb1b227f..a32387b4c778 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CompactionCommand.java
@@ -45,7 +45,6 @@
import org.apache.hudi.table.action.compact.OperationResult;
import org.apache.hudi.utilities.UtilHelpers;
-import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.launcher.SparkLauncher;
@@ -57,6 +56,7 @@
import org.springframework.shell.standard.ShellOption;
import java.io.IOException;
+import java.io.InputStream;
import java.io.ObjectInputStream;
import java.util.ArrayList;
import java.util.HashMap;
@@ -437,15 +437,15 @@ private static String getTmpSerializerFile() {
private T deSerializeOperationResult(String inputP, FileSystem fs) throws Exception {
Path inputPath = new Path(inputP);
- FSDataInputStream fsDataInputStream = fs.open(inputPath);
- ObjectInputStream in = new ObjectInputStream(fsDataInputStream);
+ InputStream inputStream = fs.open(inputPath);
+ ObjectInputStream in = new ObjectInputStream(inputStream);
try {
T result = (T) in.readObject();
LOG.info("Result : " + result);
return result;
} finally {
in.close();
- fsDataInputStream.close();
+ inputStream.close();
}
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
index fedc2712d4c9..b196c62d0fba 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
@@ -30,7 +30,6 @@
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
import org.apache.hudi.cli.HoodieCLI;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
@@ -44,6 +43,9 @@
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
+import org.apache.hudi.storage.HoodieLocation;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.shell.standard.ShellComponent;
@@ -97,7 +99,7 @@ public String exportInstants(
List nonArchivedInstants = timeline.getInstants();
// Archived instants are in the commit archive files
- FileStatus[] statuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
+ FileStatus[] statuses = HadoopFSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
List archivedStatuses = Arrays.stream(statuses).sorted((f1, f2) -> (int) (f1.getModificationTime() - f2.getModificationTime())).collect(Collectors.toList());
if (descending) {
@@ -119,7 +121,7 @@ public String exportInstants(
private int copyArchivedInstants(List statuses, Set actionSet, int limit, String localFolder) throws Exception {
int copyCount = 0;
- FileSystem fileSystem = FSUtils.getFs(HoodieCLI.getTableMetaClient().getBasePath(), HoodieCLI.conf);
+ FileSystem fileSystem = HadoopFSUtils.getFs(HoodieCLI.getTableMetaClient().getBasePath(), HoodieCLI.conf);
for (FileStatus fs : statuses) {
// read the archived file
@@ -167,7 +169,7 @@ private int copyArchivedInstants(List statuses, Set actionSe
LOG.error("Could not load metadata for action " + action + " at instant time " + instantTime);
continue;
}
- final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action;
+ final String outPath = localFolder + HoodieLocation.SEPARATOR + instantTime + "." + action;
writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true));
}
}
@@ -189,7 +191,7 @@ private int copyNonArchivedInstants(List instants, int limit, Str
final HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
final HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
for (HoodieInstant instant : instants) {
- String localPath = localFolder + Path.SEPARATOR + instant.getFileName();
+ String localPath = localFolder + HoodieLocation.SEPARATOR + instant.getFileName();
byte[] data = null;
switch (instant.getAction()) {
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
index 58eff5f7b31c..46a9e787ea6e 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
@@ -43,7 +43,7 @@
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
-import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.fs.CachingPath;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.avro.Schema;
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
index 281ab3994f75..742540d0ff5b 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
@@ -26,7 +26,6 @@
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieRecord;
@@ -44,6 +43,7 @@
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieSavepointException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
import org.apache.hudi.table.HoodieSparkTable;
@@ -437,7 +437,7 @@ private static int cluster(JavaSparkContext jsc, String basePath, String tableNa
private static int deduplicatePartitionPath(JavaSparkContext jsc, String duplicatedPartitionPath,
String repairedOutputPath, String basePath, boolean dryRun, String dedupeType) {
DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc),
- FSUtils.getFs(basePath, jsc.hadoopConfiguration()), DeDupeType.withName(dedupeType));
+ HadoopFSUtils.getFs(basePath, jsc.hadoopConfiguration()), DeDupeType.withName(dedupeType));
job.fixDuplicates(dryRun);
return 0;
}
@@ -469,7 +469,7 @@ public static int renamePartition(JavaSparkContext jsc, String basePath, String
// after re-writing, we can safely delete older partition.
deleteOlderPartition(basePath, oldPartition, recordsToRewrite, propsMap);
// also, we can physically delete the old partition.
- FileSystem fs = FSUtils.getFs(new Path(basePath), metaClient.getHadoopConf());
+ FileSystem fs = HadoopFSUtils.getFs(new Path(basePath), metaClient.getHadoopConf());
try {
fs.delete(new Path(basePath, oldPartition), true);
} catch (IOException e) {
@@ -555,7 +555,7 @@ private static int doBootstrap(JavaSparkContext jsc, String tableName, String ta
cfg.payloadClassName = payloadClassName;
cfg.enableHiveSync = Boolean.valueOf(enableHiveSync);
- new BootstrapExecutor(cfg, jsc, FSUtils.getFs(basePath, jsc.hadoopConfiguration()),
+ new BootstrapExecutor(cfg, jsc, HadoopFSUtils.getFs(basePath, jsc.hadoopConfiguration()),
jsc.hadoopConfiguration(), properties).execute();
return 0;
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
index 22bac81dff51..f0b653ec1e9c 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
@@ -52,6 +52,7 @@
import java.util.stream.Collectors;
import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
/**
* CLI command to display hudi table options.
@@ -261,7 +262,7 @@ private static void writeToFile(String filePath, String data) throws IOException
OutputStream os = null;
try {
os = new FileOutputStream(outFile);
- os.write(data.getBytes(), 0, data.length());
+ os.write(getUTF8Bytes(data), 0, data.length());
} finally {
os.close();
}
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
index fd09a27271a8..5726c4142d43 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
@@ -21,9 +21,9 @@
import org.apache.hudi.cli.HoodieCliSparkConfig;
import org.apache.hudi.cli.commands.SparkEnvCommand;
import org.apache.hudi.cli.commands.SparkMain;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
@@ -126,7 +126,7 @@ public static JavaSparkContext initJavaSparkContext(String name, Option
public static JavaSparkContext initJavaSparkContext(SparkConf sparkConf) {
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false);
- FSUtils.prepareHadoopConf(jsc.hadoopConfiguration());
+ HadoopFSUtils.prepareHadoopConf(jsc.hadoopConfiguration());
return jsc;
}
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java
index 1b45fdd4d072..2fc5baa70029 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCleansCommand.java
@@ -26,7 +26,6 @@
import org.apache.hudi.cli.functional.CLIFunctionalTestHarness;
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
@@ -39,6 +38,7 @@
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -93,7 +93,7 @@ public void init() throws Exception {
metaClient = HoodieCLI.getTableMetaClient();
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
- FileSystem fs = FSUtils.getFs(basePath(), hadoopConf());
+ FileSystem fs = HadoopFSUtils.getFs(basePath(), hadoopConf());
HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, tablePath);
// Create four commits
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java
index f1ea09470d35..c040d931187e 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java
@@ -25,9 +25,8 @@
import org.apache.hudi.cli.functional.CLIFunctionalTestHarness;
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
import org.apache.hudi.client.HoodieTimelineArchiver;
-import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
-import org.apache.hudi.common.fs.NoOpConsistencyGuard;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
+import org.apache.hudi.hadoop.fs.NoOpConsistencyGuard;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -44,6 +43,7 @@
import org.apache.hudi.config.HoodieCleanConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem;
import org.apache.hudi.table.HoodieSparkTable;
import org.junit.jupiter.api.BeforeEach;
@@ -166,7 +166,7 @@ private void generateCompactionInstances() throws IOException {
// so the archival in data table can happen
HoodieTestUtils.createCompactionCommitInMetadataTable(hadoopConf(),
new HoodieWrapperFileSystem(
- FSUtils.getFs(tablePath, hadoopConf()), new NoOpConsistencyGuard()), tablePath, "007");
+ HadoopFSUtils.getFs(tablePath, hadoopConf()), new NoOpConsistencyGuard()), tablePath, "007");
}
private void generateArchive() throws IOException {
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java
index c12ad676d41c..1ce777c71b35 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java
@@ -26,7 +26,6 @@
import org.apache.hudi.cli.functional.CLIFunctionalTestHarness;
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
@@ -38,6 +37,7 @@
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -92,7 +92,7 @@ public void testDiffFile() throws Exception {
HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
- FileSystem fs = FSUtils.getFs(basePath(), hadoopConf());
+ FileSystem fs = HadoopFSUtils.getFs(basePath(), hadoopConf());
HoodieTestDataGenerator.writePartitionMetadataDeprecated(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, tablePath);
// Create four commits
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
index 0f796c8195a1..8c433d842a1f 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
@@ -27,7 +27,6 @@
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
import org.apache.hudi.common.config.HoodieCommonConfig;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -44,6 +43,8 @@
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
+import org.apache.hudi.storage.HoodieLocation;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -108,7 +109,7 @@ public void init() throws IOException, InterruptedException, URISyntaxException
"", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
Files.createDirectories(Paths.get(partitionPath));
- fs = FSUtils.getFs(tablePath, hadoopConf());
+ fs = HadoopFSUtils.getFs(tablePath, hadoopConf());
try (HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionPath))
@@ -201,7 +202,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc
// write to path '2015/03/16'.
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
- partitionPath = tablePath + Path.SEPARATOR + HoodieTestCommitMetadataGenerator.DEFAULT_SECOND_PARTITION_PATH;
+ partitionPath = tablePath + HoodieLocation.SEPARATOR + HoodieTestCommitMetadataGenerator.DEFAULT_SECOND_PARTITION_PATH;
Files.createDirectories(Paths.get(partitionPath));
HoodieLogFormat.Writer writer = null;
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java
index db9e85acc844..6756ec267808 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java
@@ -38,6 +38,7 @@
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.testutils.Assertions;
@@ -105,7 +106,7 @@ public class TestRepairsCommand extends CLIFunctionalTestHarness {
public void init() throws IOException {
String tableName = tableName();
tablePath = tablePath(tableName);
- fs = FSUtils.getFs(tablePath, hadoopConf());
+ fs = HadoopFSUtils.getFs(tablePath, hadoopConf());
// Create table and connect
new TableCommand().createTable(
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java
index c1c44f625188..22d108241c6c 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java
@@ -32,6 +32,7 @@
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.storage.HoodieLocation;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
@@ -46,7 +47,6 @@
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.Collections;
@@ -55,6 +55,7 @@
import java.util.Map;
import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME;
+import static org.apache.hudi.common.util.StringUtils.fromUTF8Bytes;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -146,7 +147,7 @@ public void testCreateWithSpecifiedValues() {
assertTrue(ShellEvaluationResultUtil.isSuccess(result));
assertEquals("Metadata for table " + tableName + " loaded", result.toString());
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
- assertEquals(metaPath + Path.SEPARATOR + "archive", client.getArchivePath());
+ assertEquals(metaPath + HoodieLocation.SEPARATOR + "archive", client.getArchivePath());
assertEquals(tablePath, client.getBasePath());
assertEquals(metaPath, client.getMetaPath());
assertEquals(HoodieTableType.MERGE_ON_READ, client.getTableType());
@@ -185,7 +186,7 @@ public void testRefresh() throws IOException {
private void testRefreshCommand(String command) throws IOException {
// clean table matedata
FileSystem fs = FileSystem.get(hadoopConf());
- fs.delete(new Path(tablePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME), true);
+ fs.delete(new Path(tablePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME), true);
// Create table
assertTrue(prepareTable());
@@ -285,6 +286,6 @@ private String getFileContent(String fileToReadStr) throws IOException {
byte[] data = new byte[(int) fileToRead.length()];
fis.read(data);
fis.close();
- return new String(data, StandardCharsets.UTF_8);
+ return fromUTF8Bytes(data);
}
}
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java
index 4d1a0ec3fb74..237a9f1985be 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java
@@ -33,8 +33,6 @@
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.testutils.HoodieClientTestUtils;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
@@ -45,6 +43,8 @@
import org.junit.jupiter.params.provider.ValueSource;
import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
import java.util.Arrays;
import java.util.stream.Stream;
@@ -115,7 +115,7 @@ private static Stream testArgsForUpgradeDowngradeCommand() {
public void testUpgradeDowngradeCommand(HoodieTableVersion fromVersion, HoodieTableVersion toVersion) throws Exception {
// Start with hoodie.table.version to 5
metaClient.getTableConfig().setTableVersion(HoodieTableVersion.FIVE);
- try (FSDataOutputStream os = metaClient.getFs().create(new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE), true)) {
+ try (OutputStream os = metaClient.getFs().create(new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE), true)) {
metaClient.getTableConfig().getProps().store(os, "");
}
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
@@ -163,10 +163,10 @@ private void verifyTableVersion(HoodieTableVersion expectedVersion) throws IOExc
private void assertTableVersionFromPropertyFile(HoodieTableVersion expectedVersion) throws IOException {
Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
// Load the properties and verify
- FSDataInputStream fsDataInputStream = metaClient.getFs().open(propertyFile);
+ InputStream inputStream = metaClient.getFs().open(propertyFile);
HoodieConfig config = new HoodieConfig();
- config.getProps().load(fsDataInputStream);
- fsDataInputStream.close();
+ config.getProps().load(inputStream);
+ inputStream.close();
assertEquals(Integer.toString(expectedVersion.versionCode()), config.getString(HoodieTableConfig.VERSION));
}
}
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java
index f22ce1bbaf52..4e7a9c68a1e8 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java
@@ -18,7 +18,6 @@
package org.apache.hudi.cli.integ;
-import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.commands.TableCommand;
@@ -27,6 +26,8 @@
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.functional.TestBootstrap;
+import org.apache.hudi.storage.HoodieLocation;
+
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.BeforeEach;
@@ -64,8 +65,8 @@ public class ITTestBootstrapCommand extends HoodieCLIIntegrationTestBase {
public void init() {
String srcName = "source";
tableName = "test-table";
- sourcePath = basePath + Path.SEPARATOR + srcName;
- tablePath = basePath + Path.SEPARATOR + tableName;
+ sourcePath = basePath + HoodieLocation.SEPARATOR + srcName;
+ tablePath = basePath + HoodieLocation.SEPARATOR + tableName;
// generate test data
partitions = Arrays.asList("2018", "2019", "2020");
@@ -73,7 +74,7 @@ public void init() {
for (int i = 0; i < partitions.size(); i++) {
Dataset df = TestBootstrap.generateTestRawTripDataset(timestamp,
i * NUM_OF_RECORDS, i * NUM_OF_RECORDS + NUM_OF_RECORDS, null, jsc, sqlContext);
- df.write().parquet(sourcePath + Path.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i));
+ df.write().parquet(sourcePath + HoodieLocation.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i));
}
}
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java
index a71697657a0d..34becfa0de32 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java
@@ -18,9 +18,6 @@
package org.apache.hudi.cli.integ;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.commands.TableCommand;
import org.apache.hudi.cli.testutils.HoodieCLIIntegrationTestBase;
@@ -29,10 +26,14 @@
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.storage.HoodieLocation;
import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.hudi.utilities.HDFSParquetImporter;
import org.apache.hudi.utilities.functional.TestHDFSParquetImporter;
import org.apache.hudi.utilities.functional.TestHDFSParquetImporter.HoodieTripModel;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.Path;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.BeforeEach;
@@ -43,12 +44,14 @@
import org.springframework.shell.Shell;
import java.io.IOException;
+import java.io.OutputStream;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.text.ParseException;
import java.util.List;
import java.util.stream.Collectors;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
import static org.junit.jupiter.api.Assertions.assertAll;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -74,14 +77,14 @@ public class ITTestHDFSParquetImportCommand extends HoodieCLIIntegrationTestBase
@BeforeEach
public void init() throws IOException, ParseException {
tableName = "test_table";
- tablePath = basePath + Path.SEPARATOR + tableName;
+ tablePath = basePath + HoodieLocation.SEPARATOR + tableName;
sourcePath = new Path(basePath, "source");
targetPath = new Path(tablePath);
schemaFile = new Path(basePath, "file.schema").toString();
// create schema file
- try (FSDataOutputStream schemaFileOS = fs.create(new Path(schemaFile))) {
- schemaFileOS.write(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA.getBytes());
+ try (OutputStream schemaFileOS = fs.create(new Path(schemaFile))) {
+ schemaFileOS.write(getUTF8Bytes(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA));
}
importer = new TestHDFSParquetImporter();
@@ -106,7 +109,7 @@ public void testConvertWithInsert() throws IOException {
() -> assertEquals("Table imported to hoodie format", result.toString()));
// Check hudi table exist
- String metaPath = targetPath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
+ String metaPath = targetPath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
assertTrue(Files.exists(Paths.get(metaPath)), "Hoodie table not exist.");
// Load meta data
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java
index 5aacfd82de04..194c0b498895 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java
@@ -18,7 +18,6 @@
package org.apache.hudi.cli.integ;
-import org.apache.hadoop.fs.Path;
import org.apache.hudi.cli.commands.TableCommand;
import org.apache.hudi.cli.testutils.HoodieCLIIntegrationTestBase;
import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil;
@@ -26,6 +25,8 @@
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.FileCreateUtils;
+import org.apache.hudi.storage.HoodieLocation;
+
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
@@ -53,7 +54,7 @@ public class ITTestMarkersCommand extends HoodieCLIIntegrationTestBase {
@BeforeEach
public void init() throws IOException {
String tableName = "test_table";
- tablePath = basePath + Path.SEPARATOR + tableName;
+ tablePath = basePath + HoodieLocation.SEPARATOR + tableName;
// Create table and connect
new TableCommand().createTable(
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java
index f74d3c0adfe9..3aebd6a483ff 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java
@@ -33,6 +33,7 @@
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
+import org.apache.hudi.storage.HoodieLocation;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.BeforeEach;
@@ -65,7 +66,7 @@ public class ITTestSavepointsCommand extends HoodieCLIIntegrationTestBase {
@BeforeEach
public void init() throws IOException {
String tableName = "test_table";
- tablePath = basePath + Path.SEPARATOR + tableName;
+ tablePath = basePath + HoodieLocation.SEPARATOR + tableName;
// Create table and connect
new TableCommand().createTable(
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java
index 67592be1adcf..0a11ca3aaaf0 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java
@@ -26,13 +26,13 @@
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
+import java.io.OutputStream;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
@@ -42,6 +42,7 @@
import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName;
import static org.apache.hudi.common.util.CollectionUtils.createImmutableList;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
/**
* Class to be used in tests to keep generating test inserts and updates against a corpus.
@@ -113,8 +114,8 @@ public static void createCommitFileWithMetadata(String basePath, String commitTi
static void createFileWithMetadata(String basePath, Configuration configuration, String name, String content) throws IOException {
Path commitFilePath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + name);
- try (FSDataOutputStream os = FSUtils.getFs(basePath, configuration).create(commitFilePath, true)) {
- os.writeBytes(new String(content.getBytes(StandardCharsets.UTF_8)));
+ try (OutputStream os = HadoopFSUtils.getFs(basePath, configuration).create(commitFilePath, true)) {
+ os.write(getUTF8Bytes(content));
}
}
diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml
index 8c5d6cde7191..c21553158a83 100644
--- a/hudi-client/hudi-client-common/pom.xml
+++ b/hudi-client/hudi-client-common/pom.xml
@@ -20,12 +20,12 @@
hudi-client
org.apache.hudi
- 0.14.1
+ 0.15.0-SNAPSHOT
4.0.0
hudi-client-common
- 0.14.1
+ 0.15.0-SNAPSHOT
hudi-client-common
jar
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java
index 9236197a4802..8980f9044211 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java
@@ -25,7 +25,6 @@
import org.apache.hudi.client.transaction.TransactionManager;
import org.apache.hudi.client.utils.TransactionUtils;
import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -38,6 +37,7 @@
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieWriteConflictException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.metrics.HoodieMetrics;
import org.apache.hudi.table.HoodieTable;
@@ -61,6 +61,7 @@ public abstract class BaseHoodieClient implements Serializable, AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(BaseHoodieClient.class);
+ private static final long serialVersionUID = 1L;
protected final transient FileSystem fs;
protected final transient HoodieEngineContext context;
protected final transient Configuration hadoopConf;
@@ -85,7 +86,7 @@ protected BaseHoodieClient(HoodieEngineContext context, HoodieWriteConfig client
protected BaseHoodieClient(HoodieEngineContext context, HoodieWriteConfig clientConfig,
Option timelineServer) {
this.hadoopConf = context.getHadoopConf().get();
- this.fs = FSUtils.getFs(clientConfig.getBasePath(), hadoopConf);
+ this.fs = HadoopFSUtils.getFs(clientConfig.getBasePath(), hadoopConf);
this.context = context;
this.basePath = clientConfig.getBasePath();
this.config = clientConfig;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java
index 2da144162115..d3262ef91ca7 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java
@@ -72,7 +72,6 @@
import javax.annotation.Nullable;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
@@ -85,6 +84,7 @@
import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
import static org.apache.hudi.metadata.HoodieTableMetadata.isMetadataTable;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.isIndexingCommit;
@@ -494,13 +494,13 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata,
preCommit(metadata);
}
// Update table's metadata (table)
- writeTableMetadata(table, clusteringInstant.getTimestamp(), metadata, writeStatuses.orElse(context.emptyHoodieData()));
+ writeTableMetadata(table, clusteringInstant.getTimestamp(), metadata, writeStatuses.orElseGet(context::emptyHoodieData));
LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata);
table.getActiveTimeline().transitionReplaceInflightToComplete(
clusteringInstant,
- Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+ Option.of(getUTF8Bytes(metadata.toJsonString())));
} catch (Exception e) {
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
} finally {
@@ -1008,7 +1008,8 @@ private List getInstantsToRollbackForLazyCleanPolicy(HoodieTableMetaClie
*/
@Deprecated
public boolean rollback(final String commitInstantTime, Option pendingRollbackInfo, boolean skipLocking) throws HoodieRollbackException {
- final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime());
+ final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().getTimestamp())
+ .orElseGet(HoodieActiveTimeline::createNewInstantTime);
return rollback(commitInstantTime, pendingRollbackInfo, rollbackInstantTime, skipLocking);
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
index a62f1d042447..4a36b90ac2bf 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
@@ -93,7 +93,6 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@@ -106,6 +105,7 @@
import static org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName;
import static org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath;
/**
@@ -285,7 +285,7 @@ protected void commit(HoodieTable table, String commitActionType, String instant
// update Metadata table
writeTableMetadata(table, instantTime, metadata, writeStatuses);
activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
- Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+ Option.of(getUTF8Bytes(metadata.toJsonString())));
}
// Save internal schema
@@ -297,7 +297,7 @@ private void saveInternalSchema(HoodieTable table, String instantTime, HoodieCom
InternalSchema internalSchema;
Schema avroSchema = HoodieAvroUtils.createHoodieWriteSchema(config.getSchema(), config.allowOperationMetadataField());
if (historySchemaStr.isEmpty()) {
- internalSchema = SerDeHelper.fromJson(config.getInternalSchema()).orElse(AvroInternalSchemaConverter.convert(avroSchema));
+ internalSchema = SerDeHelper.fromJson(config.getInternalSchema()).orElseGet(() -> AvroInternalSchemaConverter.convert(avroSchema));
internalSchema.setSchemaId(Long.parseLong(instantTime));
} else {
internalSchema = InternalSchemaUtils.searchSchema(Long.parseLong(instantTime),
@@ -1542,7 +1542,7 @@ private void commitTableChange(InternalSchema newSchema, HoodieTableMetaClient m
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
metadata.setOperationType(WriteOperationType.ALTER_SCHEMA);
try {
- timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+ timeLine.transitionRequestedToInflight(requested, Option.of(getUTF8Bytes(metadata.toJsonString())));
} catch (IOException io) {
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io);
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
index 257d2cd855cc..e5ae98644c18 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
@@ -41,7 +41,7 @@
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.hadoop.CachingPath;
+import org.apache.hudi.hadoop.fs.CachingPath;
import org.apache.hudi.table.action.compact.OperationResult;
import org.apache.hadoop.fs.FileStatus;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java
index d4abfa82d59f..e08bcbf6957b 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java
@@ -25,8 +25,6 @@
import org.apache.hudi.client.utils.MetadataConversionUtils;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
-import org.apache.hudi.common.fs.StorageSchemes;
import org.apache.hudi.common.model.HoodieArchivedLogFile;
import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
import org.apache.hudi.common.model.HoodieAvroPayload;
@@ -56,7 +54,9 @@
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem;
import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.storage.StorageSchemes;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
import org.apache.hudi.table.marker.WriteMarkers;
@@ -594,11 +594,13 @@ private boolean deleteArchivedInstants(List archivedInstants, Hoo
);
}
if (!completedInstants.isEmpty()) {
- context.foreach(
- completedInstants,
- instant -> activeTimeline.deleteInstantFileIfExists(instant),
- Math.min(completedInstants.size(), config.getArchiveDeleteParallelism())
- );
+ // Due to the concurrency between deleting completed instants and reading data,
+ // there may be hole in the timeline, which can lead to errors when reading data.
+ // Therefore, the concurrency of deleting completed instants is temporarily disabled,
+ // and instants are deleted in ascending order to prevent the occurrence of such holes.
+ // See HUDI-7207 and #10325.
+ completedInstants.stream()
+ .forEach(instant -> activeTimeline.deleteInstantFileIfExists(instant));
}
return true;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
index f1290bb9cc31..1138e98e9ce2 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
@@ -20,7 +20,6 @@
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.view.FileSystemViewManager;
@@ -28,6 +27,7 @@
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.common.util.NetworkUtils;
import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.hadoop.conf.Configuration;
@@ -176,7 +176,7 @@ private void startServer(TimelineServiceCreator timelineServiceCreator) throws I
this.serviceConfig = timelineServiceConfBuilder.build();
server = timelineServiceCreator.create(context, hadoopConf.newCopy(), serviceConfig,
- FSUtils.getFs(writeConfig.getBasePath(), hadoopConf.newCopy()), viewManager);
+ HadoopFSUtils.getFs(writeConfig.getBasePath(), hadoopConf.newCopy()), viewManager);
serverPort = server.startService();
LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java
index 7c2642da250c..40e08275b29e 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java
@@ -22,6 +22,7 @@
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.storage.HoodieLocation;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
@@ -51,7 +52,7 @@ public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String
boolean deleted = false;
try {
String heartbeatFolderPath = HoodieTableMetaClient.getHeartbeatFolderPath(basePath);
- deleted = fs.delete(new Path(heartbeatFolderPath + Path.SEPARATOR + instantTime), false);
+ deleted = fs.delete(new Path(heartbeatFolderPath + HoodieLocation.SEPARATOR + instantTime), false);
if (!deleted) {
LOG.error("Failed to delete heartbeat for instant " + instantTime);
} else {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java
index 93656aa29461..bb08ae997d99 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java
@@ -22,6 +22,7 @@
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieHeartbeatException;
+import org.apache.hudi.storage.HoodieLocation;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -226,7 +227,8 @@ private void stopHeartbeatTimer(Heartbeat heartbeat) {
}
public static Boolean heartbeatExists(FileSystem fs, String basePath, String instantTime) throws IOException {
- Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + Path.SEPARATOR + instantTime);
+ Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath)
+ + HoodieLocation.SEPARATOR + instantTime);
return fs.exists(heartbeatFilePath);
}
@@ -253,7 +255,7 @@ private void updateHeartbeat(String instantTime) throws HoodieHeartbeatException
try {
Long newHeartbeatTime = System.currentTimeMillis();
OutputStream outputStream =
- this.fs.create(new Path(heartbeatFolderPath + Path.SEPARATOR + instantTime), true);
+ this.fs.create(new Path(heartbeatFolderPath + HoodieLocation.SEPARATOR + instantTime), true);
outputStream.close();
Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime);
if (heartbeat.getLastHeartbeatTime() != null && isHeartbeatExpired(instantTime)) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java
index da7e71a20580..3cd3cefe750b 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java
@@ -22,8 +22,6 @@
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.config.LockConfiguration;
import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.fs.StorageSchemes;
import org.apache.hudi.common.lock.LockProvider;
import org.apache.hudi.common.lock.LockState;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -34,9 +32,11 @@
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieLockException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
+import org.apache.hudi.storage.HoodieLocation;
+import org.apache.hudi.storage.StorageSchemes;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -44,6 +44,7 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.io.InputStream;
import java.io.Serializable;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
@@ -77,13 +78,13 @@ public FileSystemBasedLockProvider(final LockConfiguration lockConfiguration, fi
String lockDirectory = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY, null);
if (StringUtils.isNullOrEmpty(lockDirectory)) {
lockDirectory = lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key())
- + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
+ + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
}
this.lockTimeoutMinutes = lockConfiguration.getConfig().getInteger(FILESYSTEM_LOCK_EXPIRE_PROP_KEY);
- this.lockFile = new Path(lockDirectory + Path.SEPARATOR + LOCK_FILE_NAME);
+ this.lockFile = new Path(lockDirectory + HoodieLocation.SEPARATOR + LOCK_FILE_NAME);
this.lockInfo = new LockInfo();
this.sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
- this.fs = FSUtils.getFs(this.lockFile.toString(), configuration);
+ this.fs = HadoopFSUtils.getFs(this.lockFile.toString(), configuration);
List customSupportedFSs = lockConfiguration.getConfig().getStringList(HoodieCommonConfig.HOODIE_FS_ATOMIC_CREATION_SUPPORT.key(), ",", new ArrayList<>());
if (!customSupportedFSs.contains(this.fs.getScheme()) && !StorageSchemes.isAtomicCreationSupported(this.fs.getScheme())) {
throw new HoodieLockException("Unsupported scheme :" + this.fs.getScheme() + ", since this fs can not support atomic creation");
@@ -163,12 +164,10 @@ private boolean checkIfExpired() {
}
private void acquireLock() {
- try {
+ try (FSDataOutputStream fos = fs.create(this.lockFile, false)) {
if (!fs.exists(this.lockFile)) {
- FSDataOutputStream fos = fs.create(this.lockFile, false);
initLockInfo();
fos.writeBytes(lockInfo.toString());
- fos.close();
}
} catch (IOException e) {
throw new HoodieIOException(generateLogStatement(LockState.FAILED_TO_ACQUIRE), e);
@@ -182,11 +181,9 @@ public void initLockInfo() {
}
public void reloadCurrentOwnerLockInfo() {
- try {
+ try (InputStream is = fs.open(this.lockFile)) {
if (fs.exists(this.lockFile)) {
- FSDataInputStream fis = fs.open(this.lockFile);
- this.currentOwnerLockInfo = FileIOUtils.readAsUTFString(fis);
- fis.close();
+ this.currentOwnerLockInfo = FileIOUtils.readAsUTFString(is);
} else {
this.currentOwnerLockInfo = "";
}
@@ -224,6 +221,6 @@ public static TypedProperties getLockConfig(String tablePath) {
* IMPORTANT: this path should be shared especially when there is engine cooperation.
*/
private static String defaultLockPath(String tablePath) {
- return tablePath + Path.SEPARATOR + AUXILIARYFOLDER_NAME;
+ return tablePath + HoodieLocation.SEPARATOR + AUXILIARYFOLDER_NAME;
}
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java
index 598f7cd70721..663a03b79079 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java
@@ -26,6 +26,7 @@
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.config.HoodieLockConfig;
import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieLockException;
import org.apache.hadoop.fs.FileSystem;
@@ -107,7 +108,11 @@ public void lock() {
*/
public void unlock() {
getLockProvider().unlock();
- metrics.updateLockHeldTimerMetrics();
+ try {
+ metrics.updateLockHeldTimerMetrics();
+ } catch (HoodieException e) {
+ LOG.error(String.format("Exception encountered when updating lock metrics: %s", e));
+ }
close();
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java
index 31b92dcf914e..4299a603ece9 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java
@@ -31,6 +31,7 @@
import org.apache.curator.framework.recipes.locks.InterProcessMutex;
import org.apache.curator.retry.BoundedExponentialBackoffRetry;
import org.apache.hadoop.conf.Configuration;
+import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -74,8 +75,48 @@ public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, fin
.connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY, DEFAULT_ZK_CONNECTION_TIMEOUT_MS))
.build();
this.curatorFrameworkClient.start();
+ createPathIfNotExists();
}
+ private String getLockPath() {
+ return lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP_KEY) + "/"
+ + this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP_KEY);
+ }
+
+ private void createPathIfNotExists() {
+ try {
+ String lockPath = getLockPath();
+ LOG.info(String.format("Creating zookeeper path %s if not exists", lockPath));
+ String[] parts = lockPath.split("/");
+ StringBuilder currentPath = new StringBuilder();
+ for (String part : parts) {
+ if (!part.isEmpty()) {
+ currentPath.append("/").append(part);
+ createNodeIfNotExists(currentPath.toString());
+ }
+ }
+ } catch (Exception e) {
+ LOG.error("Failed to create ZooKeeper path: " + e.getMessage());
+ throw new HoodieLockException("Failed to initialize ZooKeeper path", e);
+ }
+ }
+
+ private void createNodeIfNotExists(String path) throws Exception {
+ if (this.curatorFrameworkClient.checkExists().forPath(path) == null) {
+ try {
+ this.curatorFrameworkClient.create().forPath(path);
+ // to avoid failure due to synchronous calls.
+ } catch (KeeperException e) {
+ if (e.code() == KeeperException.Code.NODEEXISTS) {
+ LOG.debug(String.format("Node already exist for path = %s", path));
+ } else {
+ throw new HoodieLockException("Failed to create zookeeper node", e);
+ }
+ }
+ }
+ }
+
+
// Only used for testing
public ZookeeperBasedLockProvider(
final LockConfiguration lockConfiguration, final CuratorFramework curatorFrameworkClient) {
@@ -85,6 +126,7 @@ public ZookeeperBasedLockProvider(
synchronized (this.curatorFrameworkClient) {
if (this.curatorFrameworkClient.getState() != CuratorFrameworkState.STARTED) {
this.curatorFrameworkClient.start();
+ createPathIfNotExists();
}
}
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java
index 1ef85f5ae358..3a6d2509ad9b 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java
@@ -20,7 +20,6 @@
package org.apache.hudi.client.utils;
-import org.apache.hudi.client.HoodieTimelineArchiver;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -51,7 +50,7 @@
*/
public class ArchivalUtils {
- private static final Logger LOG = LoggerFactory.getLogger(HoodieTimelineArchiver.class);
+ private static final Logger LOG = LoggerFactory.getLogger(ArchivalUtils.class);
/**
* getMinAndMaxInstantsToKeep is used by archival service to find the
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java
index d162fe28a62b..5f1ad9331ba8 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/TransactionUtils.java
@@ -79,7 +79,7 @@ public static Option resolveWriteConflictIfAny(
table.getMetaClient(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant),
completedInstantsDuringCurrentWriteOperation);
- final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.orElse(new HoodieCommitMetadata()));
+ final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.orElseGet(HoodieCommitMetadata::new));
instantStream.forEach(instant -> {
try {
ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient());
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java
index a129ff950903..a41141520234 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java
@@ -35,6 +35,10 @@
import java.io.IOException;
import java.util.Properties;
+import static org.apache.hudi.common.model.HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS;
+import static org.apache.hudi.common.model.HoodieCleaningPolicy.KEEP_LATEST_COMMITS;
+import static org.apache.hudi.common.model.HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS;
+
/**
* Clean related config.
*/
@@ -52,9 +56,9 @@ public class HoodieCleanConfig extends HoodieConfig {
.key("hoodie.clean.automatic")
.defaultValue("true")
.markAdvanced()
- .withDocumentation("When enabled, the cleaner table service is invoked immediately after each commit,"
- + " to delete older file slices. It's recommended to enable this, to ensure metadata and data storage"
- + " growth is bounded.");
+ .withDocumentation("When enabled, the cleaner table service is invoked immediately after each commit, "
+ + "to delete older file slices. It's recommended to enable this, to ensure metadata and data storage "
+ + "growth is bounded.");
public static final ConfigProperty ASYNC_CLEAN = ConfigProperty
.key("hoodie.clean.async")
@@ -67,7 +71,7 @@ public class HoodieCleanConfig extends HoodieConfig {
@Deprecated
public static final ConfigProperty CLEANER_POLICY = ConfigProperty
.key("hoodie.cleaner.policy")
- .defaultValue(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name())
+ .defaultValue(KEEP_LATEST_COMMITS.name())
.withDocumentation(HoodieCleaningPolicy.class)
.markAdvanced()
.withInferFunction(cfg -> {
@@ -81,13 +85,13 @@ public class HoodieCleanConfig extends HoodieConfig {
// "hoodie.cleaner.hours.retained" (inferred as KEEP_LATEST_BY_HOURS)
// "hoodie.cleaner.fileversions.retained" (inferred as KEEP_LATEST_FILE_VERSIONS)
if (isCommitsRetainedConfigured && !isHoursRetainedConfigured && !isFileVersionsRetainedConfigured) {
- return Option.of(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name());
+ return Option.of(KEEP_LATEST_COMMITS.name());
}
if (!isCommitsRetainedConfigured && isHoursRetainedConfigured && !isFileVersionsRetainedConfigured) {
- return Option.of(HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS.name());
+ return Option.of(KEEP_LATEST_BY_HOURS.name());
}
if (!isCommitsRetainedConfigured && !isHoursRetainedConfigured && isFileVersionsRetainedConfigured) {
- return Option.of(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name());
+ return Option.of(KEEP_LATEST_FILE_VERSIONS.name());
}
return Option.empty();
});
@@ -95,22 +99,23 @@ public class HoodieCleanConfig extends HoodieConfig {
public static final ConfigProperty CLEANER_COMMITS_RETAINED = ConfigProperty
.key(CLEANER_COMMITS_RETAINED_KEY)
.defaultValue("10")
- .withDocumentation("Number of commits to retain, without cleaning. This will be retained for num_of_commits * time_between_commits "
- + "(scheduled). This also directly translates into how much data retention the table supports for incremental queries.");
+ .withDocumentation("When " + KEEP_LATEST_COMMITS.name() + " cleaning policy is used, the number of commits to retain, without cleaning. "
+ + "This will be retained for num_of_commits * time_between_commits (scheduled). This also directly translates into how much "
+ + "data retention the table supports for incremental queries.");
public static final ConfigProperty CLEANER_HOURS_RETAINED = ConfigProperty.key(CLEANER_HOURS_RETAINED_KEY)
.defaultValue("24")
.markAdvanced()
- .withDocumentation("Number of hours for which commits need to be retained. This config provides a more flexible option as"
- + "compared to number of commits retained for cleaning service. Setting this property ensures all the files, but the latest in a file group,"
- + " corresponding to commits with commit times older than the configured number of hours to be retained are cleaned.");
+ .withDocumentation("When " + KEEP_LATEST_BY_HOURS.name() + " cleaning policy is used, the number of hours for which commits need to be retained. "
+ + "This config provides a more flexible option as compared to number of commits retained for cleaning service. Setting this property ensures "
+ + "all the files, but the latest in a file group, corresponding to commits with commit times older than the configured number of hours to be retained are cleaned.");
public static final ConfigProperty CLEANER_FILE_VERSIONS_RETAINED = ConfigProperty
.key(CLEANER_FILE_VERSIONS_RETAINED_KEY)
.defaultValue("3")
.markAdvanced()
- .withDocumentation("When " + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name() + " cleaning policy is used, "
- + " the minimum number of file slices to retain in each file group, during cleaning.");
+ .withDocumentation("When " + KEEP_LATEST_FILE_VERSIONS.name() + " cleaning policy is used, "
+ + "the minimum number of file slices to retain in each file group, during cleaning.");
public static final ConfigProperty CLEAN_TRIGGER_STRATEGY = ConfigProperty
.key("hoodie.clean.trigger.strategy")
@@ -129,8 +134,8 @@ public class HoodieCleanConfig extends HoodieConfig {
.defaultValue("true")
.markAdvanced()
.withDocumentation("When enabled, the plans for each cleaner service run is computed incrementally off the events "
- + " in the timeline, since the last cleaner run. This is much more efficient than obtaining listings for the full"
- + " table for each planning (even with a metadata table).");
+ + "in the timeline, since the last cleaner run. This is much more efficient than obtaining listings for the full "
+ + "table for each planning (even with a metadata table).");
public static final ConfigProperty FAILED_WRITES_CLEANER_POLICY = ConfigProperty
.key("hoodie.cleaner.policy.failed.writes")
@@ -175,9 +180,9 @@ public class HoodieCleanConfig extends HoodieConfig {
.defaultValue("false")
.markAdvanced()
.withDocumentation("When set to true, cleaner also deletes the bootstrap base file when it's skeleton base file is "
- + " cleaned. Turn this to true, if you want to ensure the bootstrap dataset storage is reclaimed over time, as the"
- + " table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap "
- + " base files are also physically deleted, to comply with data privacy enforcement processes.");
+ + "cleaned. Turn this to true, if you want to ensure the bootstrap dataset storage is reclaimed over time, as the "
+ + "table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap "
+ + "base files are also physically deleted, to comply with data privacy enforcement processes.");
/** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
index 19e2678c8ae5..1fe86b52cbce 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
@@ -65,7 +65,7 @@ public class HoodieCompactionConfig extends HoodieConfig {
.key("hoodie.log.compaction.enable")
.defaultValue("false")
.markAdvanced()
- .sinceVersion("0.14")
+ .sinceVersion("0.14.0")
.withDocumentation("By enabling log compaction through this config, log compaction will also get enabled for the metadata table.");
public static final ConfigProperty INLINE_LOG_COMPACT = ConfigProperty
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieErrorTableConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieErrorTableConfig.java
index 68e2097c33be..8ba013b00eed 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieErrorTableConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieErrorTableConfig.java
@@ -72,6 +72,12 @@ public class HoodieErrorTableConfig {
.defaultValue(false)
.withDocumentation("Records with schema mismatch with Target Schema are sent to Error Table.");
+ public static final ConfigProperty ERROR_ENABLE_VALIDATE_RECORD_CREATION = ConfigProperty
+ .key("hoodie.errortable.validate.recordcreation.enable")
+ .defaultValue(true)
+ .sinceVersion("0.14.2")
+ .withDocumentation("Records that fail to be created due to keygeneration failure or other issues will be sent to the Error Table");
+
public static final ConfigProperty ERROR_TABLE_WRITE_FAILURE_STRATEGY = ConfigProperty
.key("hoodie.errortable.write.failure.strategy")
.defaultValue(ErrorWriteFailureStrategy.ROLLBACK_COMMIT.name())
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
index 0cf1f287976c..4e1cdb9f5d3c 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
@@ -347,6 +347,20 @@ public class HoodieWriteConfig extends HoodieConfig {
.markAdvanced()
.withDocumentation("Size of in-memory buffer used for parallelizing network reads and lake storage writes.");
+ public static final ConfigProperty WRITE_BUFFER_RECORD_SAMPLING_RATE = ConfigProperty
+ .key("hoodie.write.buffer.record.sampling.rate")
+ .defaultValue(String.valueOf(64))
+ .markAdvanced()
+ .sinceVersion("1.0.0")
+ .withDocumentation("Sampling rate of in-memory buffer used to estimate object size. Higher value lead to lower CPU usage.");
+
+ public static final ConfigProperty WRITE_BUFFER_RECORD_CACHE_LIMIT = ConfigProperty
+ .key("hoodie.write.buffer.record.cache.limit")
+ .defaultValue(String.valueOf(128 * 1024))
+ .markAdvanced()
+ .sinceVersion("1.0.0")
+ .withDocumentation("Maximum queue size of in-memory buffer for parallelizing network reads and lake storage writes.");
+
public static final ConfigProperty WRITE_EXECUTOR_DISRUPTOR_BUFFER_LIMIT_BYTES = ConfigProperty
.key("hoodie.write.executor.disruptor.buffer.limit.bytes")
.defaultValue(String.valueOf(1024))
@@ -562,7 +576,8 @@ public class HoodieWriteConfig extends HoodieConfig {
public static final ConfigProperty NUM_RETRIES_ON_CONFLICT_FAILURES = ConfigProperty
.key("hoodie.write.num.retries.on.conflict.failures")
.defaultValue(0)
- .sinceVersion("0.13.0")
+ .markAdvanced()
+ .sinceVersion("0.14.0")
.withDocumentation("Maximum number of times to retry a batch on conflict failure.");
public static final ConfigProperty WRITE_SCHEMA_OVERRIDE = ConfigProperty
@@ -1321,6 +1336,14 @@ public int getWriteBufferLimitBytes() {
return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES_VALUE));
}
+ public int getWriteBufferRecordSamplingRate() {
+ return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_RECORD_SAMPLING_RATE));
+ }
+
+ public int getWriteBufferRecordCacheLimit() {
+ return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_RECORD_CACHE_LIMIT));
+ }
+
public String getWriteExecutorDisruptorWaitStrategy() {
return getStringOrDefault(WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY);
}
@@ -2067,6 +2090,10 @@ public String parquetFieldIdWriteEnabled() {
return getString(HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED);
}
+ public boolean parquetBloomFilterEnabled() {
+ return getBooleanOrDefault(HoodieStorageConfig.PARQUET_WITH_BLOOM_FILTER_ENABLED);
+ }
+
public Option getLogDataBlockFormat() {
return Option.ofNullable(getString(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT))
.map(HoodieLogBlock.HoodieLogBlockType::fromId);
@@ -2750,6 +2777,16 @@ public Builder withWriteBufferLimitBytes(int writeBufferLimit) {
return this;
}
+ public Builder withWriteBufferRecordSamplingRate(int recordSamplingRate) {
+ writeConfig.setValue(WRITE_BUFFER_RECORD_SAMPLING_RATE, String.valueOf(recordSamplingRate));
+ return this;
+ }
+
+ public Builder withWriteBufferRecordCacheLimit(int recordCacheLimit) {
+ writeConfig.setValue(WRITE_BUFFER_RECORD_CACHE_LIMIT, String.valueOf(recordCacheLimit));
+ return this;
+ }
+
public Builder withWriteExecutorDisruptorWaitStrategy(String waitStrategy) {
writeConfig.setValue(WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY, String.valueOf(waitStrategy));
return this;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
index b6db316a3b67..890bffeb5a39 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
@@ -69,6 +69,7 @@
import java.util.TreeSet;
import static java.util.stream.Collectors.toList;
+import static org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER;
import static org.apache.hudi.table.action.commit.HoodieDeleteHelper.createDeleteRecord;
/**
@@ -185,7 +186,7 @@ public static List filterKeysFromFile(Path filePath, List candid
ValidationUtils.checkArgument(FSUtils.isBaseFile(filePath));
List foundRecordKeys = new ArrayList<>();
try (HoodieFileReader fileReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO)
- .getFileReader(configuration, filePath)) {
+ .getFileReader(DEFAULT_HUDI_CONFIG_FOR_READER, configuration, filePath)) {
// Load all rowKeys from the file, to double-confirm
if (!candidateRecordKeys.isEmpty()) {
HoodieTimer timer = HoodieTimer.start();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIndexUtils.java
index f8befee9bf9e..d22e4b21a5ec 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIndexUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIndexUtils.java
@@ -19,7 +19,6 @@
package org.apache.hudi.index.bucket;
import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.model.ConsistentHashingNode;
import org.apache.hudi.common.model.HoodieConsistentHashingMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -30,9 +29,9 @@
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem;
import org.apache.hudi.table.HoodieTable;
-import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
@@ -40,6 +39,8 @@
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
@@ -53,6 +54,7 @@
import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.HASHING_METADATA_COMMIT_FILE_SUFFIX;
import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.HASHING_METADATA_FILE_SUFFIX;
import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.getTimestampFromFile;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
/**
* Utilities class for consistent bucket index metadata management.
@@ -183,10 +185,10 @@ public static boolean saveMetadata(HoodieTable table, HoodieConsistentHashingMet
HoodieWrapperFileSystem fs = table.getMetaClient().getFs();
Path dir = FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), metadata.getPartitionPath());
Path fullPath = new Path(dir, metadata.getFilename());
- try (FSDataOutputStream fsOut = fs.create(fullPath, overwrite)) {
+ try (OutputStream out = fs.create(fullPath, overwrite)) {
byte[] bytes = metadata.toBytes();
- fsOut.write(bytes);
- fsOut.close();
+ out.write(bytes);
+ out.close();
return true;
} catch (IOException e) {
LOG.warn("Failed to update bucket metadata: " + metadata, e);
@@ -208,7 +210,7 @@ private static void createCommitMarker(HoodieTable table, Path fileStatus, Path
if (fs.exists(fullPath)) {
return;
}
- FileIOUtils.createFileInPath(fs, fullPath, Option.of(StringUtils.EMPTY_STRING.getBytes()));
+ FileIOUtils.createFileInPath(fs, fullPath, Option.of(getUTF8Bytes(StringUtils.EMPTY_STRING)));
}
/***
@@ -219,11 +221,11 @@ private static void createCommitMarker(HoodieTable table, Path fileStatus, Path
* @return HoodieConsistentHashingMetadata object
*/
private static Option loadMetadataFromGivenFile(HoodieTable table, FileStatus metaFile) {
- try {
- if (metaFile == null) {
- return Option.empty();
- }
- byte[] content = FileIOUtils.readAsByteArray(table.getMetaClient().getFs().open(metaFile.getPath()));
+ if (metaFile == null) {
+ return Option.empty();
+ }
+ try (InputStream is = table.getMetaClient().getFs().open(metaFile.getPath())) {
+ byte[] content = FileIOUtils.readAsByteArray(is);
return Option.of(HoodieConsistentHashingMetadata.fromBytes(content));
} catch (FileNotFoundException e) {
return Option.empty();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
index ca081fce60f1..5d9c5ac54962 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
@@ -19,6 +19,7 @@
package org.apache.hudi.io;
import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieReaderConfig;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.TaskContextSupplier;
import org.apache.hudi.common.fs.FSUtils;
@@ -666,7 +667,8 @@ private static HoodieLogBlock getBlock(HoodieWriteConfig writeConfig,
return new HoodieAvroDataBlock(records, header, keyField);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(
- records, header, writeConfig.getHFileCompressionAlgorithm(), new Path(writeConfig.getBasePath()));
+ records, header, writeConfig.getHFileCompressionAlgorithm(), new Path(writeConfig.getBasePath()),
+ writeConfig.getBooleanOrDefault(HoodieReaderConfig.USE_NATIVE_HFILE_READER));
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(
records,
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
index 21c0059474e8..4460e29c8a43 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
@@ -460,7 +460,8 @@ public void performMergeDataValidationCheck(WriteStatus writeStatus) {
}
long oldNumWrites = 0;
- try (HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(this.recordMerger.getRecordType()).getFileReader(hoodieTable.getHadoopConf(), oldFilePath)) {
+ try (HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(this.recordMerger.getRecordType())
+ .getFileReader(config, hoodieTable.getHadoopConf(), oldFilePath)) {
oldNumWrites = reader.getTotalRecords();
} catch (IOException e) {
throw new HoodieUpsertException("Failed to check for merge data validation", e);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java
index 738688c62193..e74ab37f4b69 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java
@@ -135,6 +135,7 @@ private HoodieMergedLogRecordScanner getLogRecordScanner(FileSlice fileSlice) {
.withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
.withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
.withRecordMerger(config.getRecordMerger())
+ .withTableMetaClient(hoodieTable.getMetaClient())
.build();
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
index 28e6c0e16794..5b7985ba9795 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
@@ -71,12 +71,12 @@ protected HoodieBaseFile getLatestBaseFile() {
}
protected HoodieFileReader createNewFileReader() throws IOException {
- return HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(),
- new Path(getLatestBaseFile().getPath()));
+ return HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType())
+ .getFileReader(config, hoodieTable.getHadoopConf(), new Path(getLatestBaseFile().getPath()));
}
protected HoodieFileReader createNewFileReader(HoodieBaseFile hoodieBaseFile) throws IOException {
- return HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(),
- new Path(hoodieBaseFile.getPath()));
+ return HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType())
+ .getFileReader(config, hoodieTable.getHadoopConf(), new Path(hoodieBaseFile.getPath()));
}
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
index d6e7a8f626eb..e508e2d2b7eb 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
@@ -67,8 +67,8 @@
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.exception.TableNotFoundException;
-import org.apache.hudi.hadoop.CachingPath;
-import org.apache.hudi.hadoop.SerializablePath;
+import org.apache.hudi.hadoop.fs.CachingPath;
+import org.apache.hudi.hadoop.fs.SerializablePath;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hadoop.conf.Configuration;
@@ -507,6 +507,7 @@ private Pair> initializeRecordIndexPartition()
// Collect record keys from the files in parallel
HoodieData records = readRecordKeysFromBaseFiles(
engineContext,
+ dataWriteConfig,
partitionBaseFilePairs,
false,
dataWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism(),
@@ -864,7 +865,8 @@ public void buildMetadataPartitions(HoodieEngineContext engineContext, List writeStatus, String instantTime) {
processAndCommit(instantTime, () -> {
Map> partitionToRecordMap =
- HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, commitMetadata, instantTime, getRecordsGenerationParams());
+ HoodieTableMetadataUtil.convertMetadataToRecords(
+ engineContext, dataWriteConfig, commitMetadata, instantTime, getRecordsGenerationParams());
// Updates for record index are created by parsing the WriteStatus which is a hudi-client object. Hence, we cannot yet move this code
// to the HoodieTableMetadataUtil class in hudi-common.
@@ -880,7 +882,8 @@ public void updateFromWriteStatuses(HoodieCommitMetadata commitMetadata, HoodieD
public void update(HoodieCommitMetadata commitMetadata, HoodieData records, String instantTime) {
processAndCommit(instantTime, () -> {
Map> partitionToRecordMap =
- HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, commitMetadata, instantTime, getRecordsGenerationParams());
+ HoodieTableMetadataUtil.convertMetadataToRecords(
+ engineContext, dataWriteConfig, commitMetadata, instantTime, getRecordsGenerationParams());
HoodieData additionalUpdates = getRecordIndexAdditionalUpserts(records, commitMetadata);
partitionToRecordMap.put(MetadataPartitionType.RECORD_INDEX, records.union(additionalUpdates));
return partitionToRecordMap;
@@ -1421,6 +1424,7 @@ private HoodieData getRecordIndexReplacedRecords(HoodieReplaceComm
return readRecordKeysFromBaseFiles(
engineContext,
+ dataWriteConfig,
partitionBaseFilePairs,
true,
dataWriteConfig.getMetadataConfig().getRecordIndexMaxParallelism(),
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java
index 47ee23bcc2fb..ef088091732b 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java
@@ -18,11 +18,11 @@
package org.apache.hudi.metrics;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import com.codahale.metrics.MetricRegistry;
import org.apache.hadoop.conf.Configuration;
@@ -50,6 +50,7 @@ public class Metrics {
private final List reporters;
private final String commonMetricPrefix;
private boolean initialized = false;
+ private transient Thread shutdownThread = null;
public Metrics(HoodieWriteConfig metricConfig) {
registry = new MetricRegistry();
@@ -65,7 +66,8 @@ public Metrics(HoodieWriteConfig metricConfig) {
}
reporters.forEach(MetricsReporter::start);
- Runtime.getRuntime().addShutdownHook(new Thread(this::shutdown));
+ shutdownThread = new Thread(() -> shutdown(true));
+ Runtime.getRuntime().addShutdownHook(shutdownThread);
this.initialized = true;
}
@@ -93,7 +95,7 @@ public static synchronized void shutdownAllMetrics() {
private List addAdditionalMetricsExporters(HoodieWriteConfig metricConfig) {
List reporterList = new ArrayList<>();
List propPathList = StringUtils.split(metricConfig.getMetricReporterFileBasedConfigs(), ",");
- try (FileSystem fs = FSUtils.getFs(propPathList.get(0), new Configuration())) {
+ try (FileSystem fs = HadoopFSUtils.getFs(propPathList.get(0), new Configuration())) {
for (String propPath : propPathList) {
HoodieWriteConfig secondarySourceConfig = HoodieWriteConfig.newBuilder().fromInputStream(
fs.open(new Path(propPath))).withPath(metricConfig.getBasePath()).build();
@@ -112,16 +114,27 @@ private List addAdditionalMetricsExporters(HoodieWriteConfig me
return reporterList;
}
- public synchronized void shutdown() {
- try {
- registerHoodieCommonMetrics();
- reporters.forEach(MetricsReporter::report);
- LOG.info("Stopping the metrics reporter...");
- reporters.forEach(MetricsReporter::stop);
- } catch (Exception e) {
- LOG.warn("Error while closing reporter", e);
- } finally {
- initialized = false;
+ public void shutdown() {
+ shutdown(false);
+ }
+
+ private synchronized void shutdown(boolean fromShutdownHook) {
+ if (!fromShutdownHook) {
+ Runtime.getRuntime().removeShutdownHook(shutdownThread);
+ } else {
+ LOG.warn("Shutting down the metrics reporter from shutdown hook.");
+ }
+ if (initialized) {
+ try {
+ registerHoodieCommonMetrics();
+ reporters.forEach(MetricsReporter::report);
+ LOG.info("Stopping the metrics reporter...");
+ reporters.forEach(MetricsReporter::stop);
+ } catch (Exception e) {
+ LOG.warn("Error while closing reporter", e);
+ } finally {
+ initialized = false;
+ }
}
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
index dfa464d8af8b..cdefb1533987 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
@@ -37,8 +37,6 @@
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.engine.TaskContextSupplier;
-import org.apache.hudi.common.fs.ConsistencyGuard;
-import org.apache.hudi.common.fs.ConsistencyGuard.FileVisibility;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.fs.FailSafeConsistencyGuard;
import org.apache.hudi.common.fs.OptimisticConsistencyGuard;
@@ -69,6 +67,8 @@
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.hadoop.fs.ConsistencyGuard;
+import org.apache.hudi.hadoop.fs.ConsistencyGuard.FileVisibility;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
@@ -643,7 +643,8 @@ public void rollbackInflightClustering(HoodieInstant inflightInstant,
private void rollbackInflightInstant(HoodieInstant inflightInstant,
Function> getPendingRollbackInstantFunc) {
final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.getTimestamp()).map(entry
- -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime());
+ -> entry.getRollbackInstant().getTimestamp())
+ .orElseGet(HoodieActiveTimeline::createNewInstantTime);
scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers(),
false);
rollback(context, commitTime, inflightInstant, false, false);
@@ -658,7 +659,8 @@ private void rollbackInflightInstant(HoodieInstant inflightInstant,
*/
public void rollbackInflightLogCompaction(HoodieInstant inflightInstant, Function> getPendingRollbackInstantFunc) {
final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.getTimestamp()).map(entry
- -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime());
+ -> entry.getRollbackInstant().getTimestamp())
+ .orElseGet(HoodieActiveTimeline::createNewInstantTime);
scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers(),
false);
rollback(context, commitTime, inflightInstant, true, false);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
index 13d43040dd8a..c0683946b9bb 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
@@ -36,6 +36,7 @@
public abstract class BaseActionExecutor implements Serializable {
+ private static final long serialVersionUID = 1L;
protected final transient HoodieEngineContext context;
protected final transient Configuration hadoopConf;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java
index 55d8e4e47af5..4f4cc7d9bc7e 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java
@@ -60,7 +60,6 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.time.Instant;
import java.util.Collections;
@@ -71,6 +70,7 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE;
public abstract class BaseCommitActionExecutor
@@ -154,7 +154,7 @@ void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String insta
String commitActionType = getCommitActionType();
HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime);
activeTimeline.transitionRequestedToInflight(requested,
- Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)),
+ Option.of(getUTF8Bytes(metadata.toJsonString())),
config.shouldAllowMultiWriteOnSameInstant());
} catch (IOException io) {
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
index c1523d564e48..7fba0463292a 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
@@ -82,7 +82,7 @@ public void runMerge(HoodieTable, ?, ?, ?> table,
HoodieRecord.HoodieRecordType recordType = table.getConfig().getRecordMerger().getRecordType();
HoodieFileReader baseFileReader = HoodieFileReaderFactory
.getReaderFactory(recordType)
- .getFileReader(hadoopConf, mergeHandle.getOldFilePath());
+ .getFileReader(writeConfig, hadoopConf, mergeHandle.getOldFilePath());
HoodieFileReader bootstrapFileReader = null;
Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields();
@@ -114,7 +114,7 @@ public void runMerge(HoodieTable, ?, ?, ?> table,
Configuration bootstrapFileConfig = new Configuration(table.getHadoopConf());
bootstrapFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).newBootstrapFileReader(
baseFileReader,
- HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(bootstrapFileConfig, bootstrapFilePath),
+ HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(writeConfig, bootstrapFileConfig, bootstrapFilePath),
mergeHandle.getPartitionFields(),
mergeHandle.getPartitionValues());
recordSchema = mergeHandle.getWriterSchemaWithMetaFields();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java
index c6fa1f4f2b2e..a49f31ead6e5 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java
@@ -36,10 +36,11 @@
import org.apache.hudi.table.HoodieTable;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Set;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
+
/**
* Base class helps to perform compact.
*
@@ -83,7 +84,7 @@ public void completeInflightCompaction(HoodieTable table, String compactionCommi
try {
activeTimeline.transitionCompactionInflightToComplete(
HoodieTimeline.getCompactionInflightInstant(compactionCommitTime),
- Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+ Option.of(getUTF8Bytes(commitMetadata.toJsonString())));
} catch (IOException e) {
throw new HoodieCompactionException(
"Failed to commit " + table.getMetaClient().getBasePath() + " at time " + compactionCommitTime, e);
@@ -95,7 +96,7 @@ public void completeInflightLogCompaction(HoodieTable table, String logCompactio
try {
activeTimeline.transitionLogCompactionInflightToComplete(
HoodieTimeline.getLogCompactionInflightInstant(logCompactionCommitTime),
- Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+ Option.of(getUTF8Bytes(commitMetadata.toJsonString())));
} catch (IOException e) {
throw new HoodieCompactionException(
"Failed to commit " + table.getMetaClient().getBasePath() + " at time " + logCompactionCommitTime, e);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
index 906ea6473a4b..d1d69be16dcf 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
@@ -207,6 +207,7 @@ public List compact(HoodieCompactionHandler compactionHandler,
.withOptimizedLogBlocksScan(executionHelper.enableOptimizedLogBlockScan(config))
.withRecordMerger(config.getRecordMerger())
.withInstantRange(instantRange)
+ .withTableMetaClient(metaClient)
.build();
Option oldDataFileOpt =
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java
index 2b7047265802..7cc0e338bcf9 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java
@@ -98,6 +98,7 @@ private boolean isFileSliceEligibleForLogCompaction(FileSlice fileSlice, String
.withBufferSize(writeConfig.getMaxDFSStreamBufferSize())
.withOptimizedLogBlocksScan(true)
.withRecordMerger(writeConfig.getRecordMerger())
+ .withTableMetaClient(metaClient)
.build();
scanner.scan(true);
int totalBlocks = scanner.getCurrentInstantLogBlocks().size();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java
index a95b3a3dc5c3..94473e98d79c 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java
@@ -56,6 +56,7 @@
*/
public class BaseRollbackHelper implements Serializable {
+ private static final long serialVersionUID = 1L;
private static final Logger LOG = LoggerFactory.getLogger(BaseRollbackHelper.class);
protected static final String EMPTY_STRING = "";
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java
index 820e998c368f..a622c5ae4334 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java
@@ -21,7 +21,6 @@
import org.apache.hudi.avro.model.HoodieRollbackRequest;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
@@ -34,6 +33,7 @@
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
+import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.FileStatus;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
index 29da31b478cb..1e0330a4defc 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
@@ -90,7 +90,7 @@ public HoodieSavepointMetadata execute() {
} catch (IOException e) {
throw new HoodieSavepointException("Failed to savepoint " + instantTime, e);
}
- }).orElse(table.getCompletedCommitsTimeline().firstInstant().get().getTimestamp());
+ }).orElseGet(() -> table.getCompletedCommitsTimeline().firstInstant().get().getTimestamp());
// Cannot allow savepoint time on a commit that could have been cleaned
ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.GREATER_THAN_OR_EQUALS, lastCommitRetained),
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java
index f9c30ca17367..a540c21a8a78 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectWriteMarkers.java
@@ -33,6 +33,7 @@
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.FileStatus;
@@ -113,7 +114,7 @@ public Set createdAndMergedDataPaths(HoodieEngineContext context, int pa
context.setJobStatus(this.getClass().getSimpleName(), "Obtaining marker files for all created, merged paths");
dataFiles.addAll(context.flatMap(subDirectories, directory -> {
Path path = new Path(directory);
- FileSystem fileSystem = FSUtils.getFs(path, serializedConf.get());
+ FileSystem fileSystem = HadoopFSUtils.getFs(path, serializedConf.get());
RemoteIterator itr = fileSystem.listFiles(path, true);
List result = new ArrayList<>();
while (itr.hasNext()) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java
index 038d21ae05c1..7c85a5a18058 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleDirectMarkerBasedDetectionStrategy.java
@@ -19,13 +19,13 @@
package org.apache.hudi.table.marker;
import org.apache.hudi.common.conflict.detection.DirectMarkerBasedDetectionStrategy;
-import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.MarkerUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieEarlyConflictDetectionException;
import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java
index b22fff750c8f..f17f166656c6 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java
@@ -19,10 +19,10 @@
package org.apache.hudi.table.marker;
import org.apache.hudi.client.transaction.DirectMarkerTransactionManager;
-import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieEarlyConflictDetectionException;
+import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java
index 7a8234c8d8a6..70cecf475d84 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java
@@ -18,10 +18,10 @@
package org.apache.hudi.table.marker;
-import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.fs.StorageSchemes;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
+import org.apache.hudi.storage.StorageSchemes;
import org.apache.hudi.table.HoodieTable;
import org.slf4j.Logger;
@@ -52,7 +52,7 @@ public static WriteMarkers get(MarkerType markerType, HoodieTable table, String
}
String basePath = table.getMetaClient().getBasePath();
if (StorageSchemes.HDFS.getScheme().equals(
- FSUtils.getFs(basePath, table.getContext().getHadoopConf().newCopy()).getScheme())) {
+ HadoopFSUtils.getFs(basePath, table.getContext().getHadoopConf().newCopy()).getScheme())) {
LOG.warn("Timeline-server-based markers are not supported for HDFS: "
+ "base path " + basePath + ". Falling back to direct markers.");
return new DirectWriteMarkers(table, instantTime);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToFiveUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToFiveUpgradeHandler.java
index 4d7c5b8b6df6..2adddf36df50 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToFiveUpgradeHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToFiveUpgradeHandler.java
@@ -77,7 +77,7 @@ public Map upgrade(HoodieWriteConfig config, HoodieEngin
private boolean hasDefaultPartitionPath(HoodieWriteConfig config, HoodieTable table) throws IOException {
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
- if (!tableConfig.getPartitionFields().isPresent()) {
+ if (!tableConfig.isTablePartitioned()) {
return false;
}
String checkPartitionPath = DEPRECATED_DEFAULT_PARTITION_PATH;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java
index cb0fca5ffee0..34d671a7cf0b 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java
@@ -29,6 +29,7 @@
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.marker.DirectWriteMarkers;
@@ -92,7 +93,7 @@ private void convertToDirectMarkers(final String commitInstantTime,
HoodieEngineContext context,
int parallelism) throws IOException {
String markerDir = table.getMetaClient().getMarkerFolderPath(commitInstantTime);
- FileSystem fileSystem = FSUtils.getFs(markerDir, context.getHadoopConf().newCopy());
+ FileSystem fileSystem = HadoopFSUtils.getFs(markerDir, context.getHadoopConf().newCopy());
Option markerTypeOption = MarkerUtils.readMarkerType(fileSystem, markerDir);
if (markerTypeOption.isPresent()) {
switch (markerTypeOption.get()) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java
index 49e83733adf0..79bdcfe80d46 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java
@@ -48,8 +48,8 @@ public static HoodieExecutor create(HoodieWriteConfig config,
ExecutorType executorType = config.getExecutorType();
switch (executorType) {
case BOUNDED_IN_MEMORY:
- return new BoundedInMemoryExecutor<>(config.getWriteBufferLimitBytes(), inputItr, consumer,
- transformFunction, preExecuteRunnable);
+ return new BoundedInMemoryExecutor<>(config.getWriteBufferLimitBytes(), config.getWriteBufferRecordSamplingRate(), config.getWriteBufferRecordCacheLimit(),
+ inputItr, consumer, transformFunction, preExecuteRunnable);
case DISRUPTOR:
return new DisruptorExecutor<>(config.getWriteExecutorDisruptorWriteBufferLimitBytes(), inputItr, consumer,
transformFunction, config.getWriteExecutorDisruptorWaitStrategy(), preExecuteRunnable);
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java
index ae8bb416c9f2..9c86cdeee811 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java
@@ -28,16 +28,16 @@
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.collection.ImmutablePair;
import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
+import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -46,6 +46,7 @@
import java.util.UUID;
import static org.apache.hudi.common.table.log.HoodieLogFormat.DEFAULT_WRITE_TOKEN;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
public class HoodieTestCommitGenerator {
public static final String BASE_FILE_WRITE_TOKEN = "1-0-1";
@@ -162,15 +163,15 @@ public static void createCommitFileWithMetadata(
String basePath, Configuration configuration,
String filename, String content) throws IOException {
Path commitFilePath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + filename);
- try (FSDataOutputStream os = FSUtils.getFs(basePath, configuration).create(commitFilePath, true)) {
- os.writeBytes(new String(content.getBytes(StandardCharsets.UTF_8)));
+ try (OutputStream os = HadoopFSUtils.getFs(basePath, configuration).create(commitFilePath, true)) {
+ os.write(getUTF8Bytes(content));
}
}
public static void createDataFile(
String basePath, Configuration configuration,
String partitionPath, String filename) throws IOException {
- FileSystem fs = FSUtils.getFs(basePath, configuration);
+ FileSystem fs = HadoopFSUtils.getFs(basePath, configuration);
Path filePath = new Path(new Path(basePath, partitionPath), filename);
Path parent = filePath.getParent();
if (!fs.exists(parent)) {
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java
index 97ad050e7240..9488d5bab6cc 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java
@@ -22,10 +22,10 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.config.LockConfiguration;
-import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.lock.LockProvider;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieLockException;
+import org.apache.hudi.hadoop.fs.HadoopFSUtils;
import java.io.IOException;
import java.io.Serializable;
@@ -56,7 +56,7 @@ public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfigur
this.retryWaitTimeMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY);
this.retryMaxCount = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY);
this.lockFile = new Path(lockDirectory + "/" + LOCK);
- this.fs = FSUtils.getFs(this.lockFile.toString(), configuration);
+ this.fs = HadoopFSUtils.getFs(this.lockFile.toString(), configuration);
}
@Override
diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/InProcessLockProviderWithRuntimeError.java
similarity index 53%
rename from hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/InProcessLockProviderWithRuntimeError.java
index c0d83e6096e3..f825012f1312 100644
--- a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/InProcessLockProviderWithRuntimeError.java
@@ -7,7 +7,7 @@
* "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
+ * 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,
@@ -16,17 +16,28 @@
* limitations under the License.
*/
-package org.apache.hudi.adapter;
+package org.apache.hudi.client.transaction;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.client.transaction.lock.InProcessLockProvider;
+import org.apache.hudi.common.config.LockConfiguration;
+
+public class InProcessLockProviderWithRuntimeError extends InProcessLockProvider {
+
+ public InProcessLockProviderWithRuntimeError(
+ LockConfiguration lockConfiguration,
+ Configuration conf) {
+ super(lockConfiguration, conf);
+ }
+
+ @Override
+ public boolean tryLock(long time, TimeUnit unit) {
+ throw new RuntimeException();
+ }
-/**
- * Adapter clazz for {@link Output}.
- */
-public interface OutputAdapter extends Output {
@Override
- default void emitWatermarkStatus(WatermarkStatus watermarkStatus) {
- // no operation
+ public void unlock() {
+ return;
}
}
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java
index 4222754a1949..c0fb8de8691f 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java
@@ -29,15 +29,19 @@
import org.apache.hudi.config.HoodieCleanConfig;
import org.apache.hudi.config.HoodieLockConfig;
import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.config.metrics.HoodieMetricsConfig;
import org.apache.hudi.exception.HoodieLockException;
+import org.apache.hudi.metrics.MetricsReporterType;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import org.junit.jupiter.api.TestInfo;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -47,14 +51,14 @@ public class TestTransactionManager extends HoodieCommonTestHarness {
TransactionManager transactionManager;
@BeforeEach
- private void init() throws IOException {
+ private void init(TestInfo testInfo) throws IOException {
initPath();
initMetaClient();
- this.writeConfig = getWriteConfig();
+ this.writeConfig = getWriteConfig(testInfo.getTags().contains("useLockProviderWithRuntimeError"));
this.transactionManager = new TransactionManager(this.writeConfig, this.metaClient.getFs());
}
- private HoodieWriteConfig getWriteConfig() {
+ private HoodieWriteConfig getWriteConfig(boolean useLockProviderWithRuntimeError) {
return HoodieWriteConfig.newBuilder()
.withPath(basePath)
.withCleanConfig(HoodieCleanConfig.newBuilder()
@@ -62,13 +66,15 @@ private HoodieWriteConfig getWriteConfig() {
.build())
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
.withLockConfig(HoodieLockConfig.newBuilder()
- .withLockProvider(InProcessLockProvider.class)
+ .withLockProvider(useLockProviderWithRuntimeError ? InProcessLockProviderWithRuntimeError.class : InProcessLockProvider.class)
.withLockWaitTimeInMillis(50L)
.withNumRetries(2)
.withRetryWaitTimeInMillis(10L)
.withClientNumRetries(2)
.withClientRetryWaitTimeInMillis(10L)
.build())
+ .forTable("testtable")
+ .withMetricsConfig(HoodieMetricsConfig.newBuilder().withReporterType(MetricsReporterType.INMEMORY.toString()).withLockingMetrics(true).on(true).build())
.build();
}
@@ -245,6 +251,19 @@ public void testTransactionsWithInstantTime() {
Assertions.assertFalse(transactionManager.getLastCompletedTransactionOwner().isPresent());
}
+ @Test
+ @Tag("useLockProviderWithRuntimeError")
+ public void testTransactionsWithUncheckedLockProviderRuntimeException() {
+ assertThrows(RuntimeException.class, () -> {
+ try {
+ transactionManager.beginTransaction(Option.empty(), Option.empty());
+ } finally {
+ transactionManager.endTransaction(Option.empty());
+ }
+ });
+
+ }
+
private Option getInstant(String timestamp) {
return Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, timestamp));
}
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java
deleted file mode 100644
index 0d2eefa08637..000000000000
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java
+++ /dev/null
@@ -1,479 +0,0 @@
-/*
- * 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.io.storage;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CellComparatorImpl;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
-import org.apache.hudi.common.engine.TaskContextSupplier;
-import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
-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.util.FileIOUtils;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieWriteConfig;
-
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.Arguments;
-import org.junit.jupiter.params.provider.MethodSource;
-import org.junit.jupiter.params.provider.ValueSource;
-import org.mockito.Mockito;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.Spliterator;
-import java.util.Spliterators;
-import java.util.TreeMap;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-import java.util.stream.Stream;
-import java.util.stream.StreamSupport;
-
-import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
-import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
-import static org.apache.hudi.common.util.CollectionUtils.toStream;
-import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR;
-import static org.apache.hudi.io.storage.HoodieAvroHFileReader.SCHEMA_KEY;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.mockito.Mockito.when;
-
-public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
- private static final String DUMMY_BASE_PATH = "dummy_base_path";
- // Number of records in HFile fixtures for compatibility tests
- private static final int NUM_RECORDS_FIXTURE = 50;
- private static final String SIMPLE_SCHEMA_HFILE_SUFFIX = "_simple.hfile";
- private static final String COMPLEX_SCHEMA_HFILE_SUFFIX = "_complex.hfile";
- private static final String BOOTSTRAP_INDEX_HFILE_SUFFIX = "_bootstrap_index_partitions.hfile";
-
- @Override
- protected Path getFilePath() {
- return new Path(tempDir.toString() + "/f1_1-0-1_000.hfile");
- }
-
- @Override
- protected HoodieAvroHFileWriter createWriter(
- Schema avroSchema, boolean populateMetaFields) throws Exception {
- String instantTime = "000";
- HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder()
- .withPath(DUMMY_BASE_PATH)
- .withIndexConfig(HoodieIndexConfig.newBuilder()
- .bloomFilterNumEntries(1000).bloomFilterFPP(0.00001).build())
- .withPopulateMetaFields(populateMetaFields)
- .build();
- Configuration conf = new Configuration();
- TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class);
- Supplier partitionSupplier = Mockito.mock(Supplier.class);
- when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier);
- when(partitionSupplier.get()).thenReturn(10);
-
- return (HoodieAvroHFileWriter)HoodieFileWriterFactory.getFileWriter(
- instantTime, getFilePath(), conf, writeConfig.getStorageConfig(), avroSchema, mockTaskContextSupplier, writeConfig.getRecordMerger().getRecordType());
- }
-
- @Override
- protected HoodieAvroFileReader createReader(
- Configuration conf) throws Exception {
- CacheConfig cacheConfig = new CacheConfig(conf);
- return new HoodieAvroHFileReader(conf, getFilePath(), cacheConfig, getFilePath().getFileSystem(conf), Option.empty());
- }
-
- @Override
- protected void verifyMetadata(Configuration conf) throws IOException {
- FileSystem fs = getFilePath().getFileSystem(conf);
- HFile.Reader hfileReader = HoodieHFileUtils.createHFileReader(fs, getFilePath(), new CacheConfig(conf), conf);
- assertEquals(HFILE_COMPARATOR.getClass(), hfileReader.getComparator().getClass());
- assertEquals(NUM_RECORDS, hfileReader.getEntries());
- }
-
- @Override
- protected void verifySchema(Configuration conf, String schemaPath) throws IOException {
- FileSystem fs = getFilePath().getFileSystem(conf);
- HFile.Reader hfileReader = HoodieHFileUtils.createHFileReader(fs, getFilePath(), new CacheConfig(conf), conf);
- assertEquals(getSchemaFromResource(TestHoodieHFileReaderWriter.class, schemaPath),
- new Schema.Parser().parse(new String(hfileReader.getHFileInfo().get(SCHEMA_KEY.getBytes()))));
- }
-
- private static Stream populateMetaFieldsAndTestAvroWithMeta() {
- return Arrays.stream(new Boolean[][] {
- {true, true},
- {false, true},
- {true, false},
- {false, false}
- }).map(Arguments::of);
- }
-
- @ParameterizedTest
- @MethodSource("populateMetaFieldsAndTestAvroWithMeta")
- public void testWriteReadHFileWithMetaFields(boolean populateMetaFields, boolean testAvroWithMeta) throws Exception {
- Schema avroSchema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchemaWithMetaFields.avsc");
- HoodieAvroHFileWriter writer = createWriter(avroSchema, populateMetaFields);
- List keys = new ArrayList<>();
- Map recordMap = new TreeMap<>();
- for (int i = 0; i < 100; i++) {
- GenericRecord record = new GenericData.Record(avroSchema);
- String key = String.format("%s%04d", "key", i);
- record.put("_row_key", key);
- keys.add(key);
- record.put("time", Integer.toString(RANDOM.nextInt()));
- record.put("number", i);
- if (testAvroWithMeta) {
- // payload does not matter. GenericRecord passed in is what matters
- writer.writeAvroWithMetadata(new HoodieAvroRecord(new HoodieKey((String) record.get("_row_key"),
- Integer.toString((Integer) record.get("number"))), new EmptyHoodieRecordPayload()).getKey(), record);
- // only HoodieKey will be looked up from the 2nd arg(HoodieRecord).
- } else {
- writer.writeAvro(key, record);
- }
- recordMap.put(key, record);
- }
- writer.close();
-
- Configuration conf = new Configuration();
- HoodieAvroHFileReader hoodieHFileReader = (HoodieAvroHFileReader) createReader(conf);
- List records = HoodieAvroHFileReader.readAllRecords(hoodieHFileReader);
- assertEquals(new ArrayList<>(recordMap.values()), records);
-
- hoodieHFileReader.close();
-
- for (int i = 0; i < 2; i++) {
- int randomRowstoFetch = 5 + RANDOM.nextInt(10);
- Set rowsToFetch = getRandomKeys(randomRowstoFetch, keys);
-
- List rowsList = new ArrayList<>(rowsToFetch);
- Collections.sort(rowsList);
-
- List expectedRecords = rowsList.stream().map(recordMap::get).collect(Collectors.toList());
-
- hoodieHFileReader = (HoodieAvroHFileReader) createReader(conf);
- List result = HoodieAvroHFileReader.readRecords(hoodieHFileReader, rowsList).stream().map(r -> (GenericRecord)r).collect(Collectors.toList());
-
- assertEquals(expectedRecords, result);
-
- result.forEach(entry -> {
- if (populateMetaFields && testAvroWithMeta) {
- assertNotNull(entry.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
- } else {
- assertNull(entry.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
- }
- });
- hoodieHFileReader.close();
- }
- }
-
- @Disabled("Disable the test with evolved schema for HFile since it's not supported")
- @ParameterizedTest
- @Override
- public void testWriteReadWithEvolvedSchema(String evolvedSchemaPath) throws Exception {
- // TODO(HUDI-3683): fix the schema evolution for HFile
- }
-
- @Test
- public void testReadHFileFormatRecords() throws Exception {
- writeFileWithSimpleSchema();
- FileSystem fs = FSUtils.getFs(getFilePath().toString(), new Configuration());
- byte[] content = FileIOUtils.readAsByteArray(
- fs.open(getFilePath()), (int) fs.getFileStatus(getFilePath()).getLen());
- // Reading byte array in HFile format, without actual file path
- Configuration hadoopConf = fs.getConf();
- HoodieAvroHFileReader hfileReader =
- new HoodieAvroHFileReader(hadoopConf, new Path(DUMMY_BASE_PATH), new CacheConfig(hadoopConf), fs, content, Option.empty());
- Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
- assertEquals(NUM_RECORDS, hfileReader.getTotalRecords());
- verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
- }
-
- @Test
- public void testReaderGetRecordIterator() throws Exception {
- writeFileWithSimpleSchema();
- HoodieAvroHFileReader hfileReader =
- (HoodieAvroHFileReader) createReader(new Configuration());
- List keys =
- IntStream.concat(IntStream.range(40, NUM_RECORDS * 2), IntStream.range(10, 20))
- .mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toList());
- Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
- Iterator> iterator = hfileReader.getRecordsByKeysIterator(keys, avroSchema);
-
- List expectedIds =
- IntStream.concat(IntStream.range(40, NUM_RECORDS), IntStream.range(10, 20))
- .boxed().collect(Collectors.toList());
- int index = 0;
- while (iterator.hasNext()) {
- GenericRecord record = (GenericRecord) iterator.next().getData();
- String key = "key" + String.format("%02d", expectedIds.get(index));
- assertEquals(key, record.get("_row_key").toString());
- assertEquals(Integer.toString(expectedIds.get(index)), record.get("time").toString());
- assertEquals(expectedIds.get(index), record.get("number"));
- index++;
- }
- }
-
- @Test
- public void testReaderGetRecordIteratorByKeys() throws Exception {
- writeFileWithSimpleSchema();
- HoodieAvroHFileReader hfileReader =
- (HoodieAvroHFileReader) createReader(new Configuration());
-
- Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
-
- List keys = Collections.singletonList("key");
- Iterator iterator =
- hfileReader.getIndexedRecordsByKeysIterator(keys, avroSchema);
-
- List recordsByKeys = toStream(iterator).map(r -> (GenericRecord) r).collect(Collectors.toList());
-
- List allRecords = toStream(hfileReader.getRecordIterator())
- .map(r -> (GenericRecord) r.getData()).collect(Collectors.toList());
-
- // no entries should match since this is exact match.
- assertEquals(Collections.emptyList(), recordsByKeys);
-
- // filter for "key00001, key05, key12, key24, key16, key2, key31, key49, key61, key50". Valid entries should be matched.
- // key00001 should not match.
- // even though key16 exists, its not in the sorted order of keys passed in. So, will not return the matched entry.
- // key2 : we don't have an exact match
- // key61 is greater than max key.
- // again, by the time we reach key50, cursor is at EOF. So no entries will be returned.
- List expectedKey1s = allRecords.stream().filter(entry -> (
- (entry.get("_row_key").toString()).contains("key05")
- || (entry.get("_row_key").toString()).contains("key12")
- || (entry.get("_row_key").toString()).contains("key24")
- || (entry.get("_row_key").toString()).contains("key31")
- || (entry.get("_row_key").toString()).contains("key49"))).collect(Collectors.toList());
- iterator =
- hfileReader.getIndexedRecordsByKeysIterator(Arrays.asList("key00001", "key05", "key12", "key24", "key16", "key31", "key49","key61","key50"), avroSchema);
- recordsByKeys =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord) r)
- .collect(Collectors.toList());
- assertEquals(expectedKey1s, recordsByKeys);
- }
-
- @Test
- public void testReaderGetRecordIteratorByKeyPrefixes() throws Exception {
- writeFileWithSimpleSchema();
- HoodieAvroHFileReader hfileReader =
- (HoodieAvroHFileReader) createReader(new Configuration());
-
- Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
-
- List keyPrefixes = Collections.singletonList("key");
- Iterator iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(keyPrefixes, avroSchema);
-
- List recordsByPrefix = toStream(iterator).map(r -> (GenericRecord)r).collect(Collectors.toList());
-
- List allRecords = toStream(hfileReader.getRecordIterator())
- .map(r -> (GenericRecord) r.getData()).collect(Collectors.toList());
-
- assertEquals(allRecords, recordsByPrefix);
-
- // filter for "key1" : entries from key10 to key19 should be matched
- List expectedKey1s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key1")).collect(Collectors.toList());
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key1"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- assertEquals(expectedKey1s, recordsByPrefix);
-
- // exact match
- List expectedKey25 = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key25")).collect(Collectors.toList());
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key25"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- assertEquals(expectedKey25, recordsByPrefix);
-
- // no match. key prefix is beyond entries in file.
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key99"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- assertEquals(Collections.emptyList(), recordsByPrefix);
-
- // no match. but keyPrefix is in between the entries found in file.
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key1234"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- assertEquals(Collections.emptyList(), recordsByPrefix);
-
- // filter for "key1", "key30" and "key60" : entries from 'key10 to key19' and 'key30' should be matched.
- List expectedKey50and1s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key1")
- || (entry.get("_row_key").toString()).contains("key30")).collect(Collectors.toList());
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key1", "key30","key6"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- assertEquals(expectedKey50and1s, recordsByPrefix);
-
- // filter for "key50" and "key0" : entries from key50 and 'key00 to key09' should be matched.
- List expectedKey50and0s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key0")
- || (entry.get("_row_key").toString()).contains("key50")).collect(Collectors.toList());
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key0", "key50"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- assertEquals(expectedKey50and0s, recordsByPrefix);
-
- // filter for "key1" and "key0" : entries from 'key10 to key19' and 'key00 to key09' should be matched.
- List expectedKey1sand0s = allRecords.stream()
- .filter(entry -> (entry.get("_row_key").toString()).contains("key1") || (entry.get("_row_key").toString()).contains("key0"))
- .collect(Collectors.toList());
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key0", "key1"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- Collections.sort(recordsByPrefix, new Comparator() {
- @Override
- public int compare(GenericRecord o1, GenericRecord o2) {
- return o1.get("_row_key").toString().compareTo(o2.get("_row_key").toString());
- }
- });
- assertEquals(expectedKey1sand0s, recordsByPrefix);
-
- // We expect the keys to be looked up in sorted order. If not, matching entries may not be returned.
- // key1 should have matching entries, but not key0.
- iterator =
- hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key1", "key0"), avroSchema);
- recordsByPrefix =
- StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
- .map(r -> (GenericRecord)r)
- .collect(Collectors.toList());
- Collections.sort(recordsByPrefix, new Comparator() {
- @Override
- public int compare(GenericRecord o1, GenericRecord o2) {
- return o1.get("_row_key").toString().compareTo(o2.get("_row_key").toString());
- }
- });
- assertEquals(expectedKey1s, recordsByPrefix);
- }
-
- @ParameterizedTest
- @ValueSource(strings = {
- "/hudi_0_9_hbase_1_2_3", "/hudi_0_10_hbase_1_2_3", "/hudi_0_11_hbase_2_4_9"})
- public void testHoodieHFileCompatibility(String hfilePrefix) throws IOException {
- // This fixture is generated from TestHoodieReaderWriterBase#testWriteReadPrimitiveRecord()
- // using different Hudi releases
- String simpleHFile = hfilePrefix + SIMPLE_SCHEMA_HFILE_SUFFIX;
- // This fixture is generated from TestHoodieReaderWriterBase#testWriteReadComplexRecord()
- // using different Hudi releases
- String complexHFile = hfilePrefix + COMPLEX_SCHEMA_HFILE_SUFFIX;
- // This fixture is generated from TestBootstrapIndex#testBootstrapIndex()
- // using different Hudi releases. The file is copied from .hoodie/.aux/.bootstrap/.partitions/
- String bootstrapIndexFile = hfilePrefix + BOOTSTRAP_INDEX_HFILE_SUFFIX;
-
- FileSystem fs = FSUtils.getFs(getFilePath().toString(), new Configuration());
- byte[] content = readHFileFromResources(simpleHFile);
- verifyHFileReader(
- HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content),
- hfilePrefix, true, HFILE_COMPARATOR.getClass(), NUM_RECORDS_FIXTURE);
-
- Configuration hadoopConf = fs.getConf();
- HoodieAvroHFileReader hfileReader =
- new HoodieAvroHFileReader(hadoopConf, new Path(DUMMY_BASE_PATH), new CacheConfig(hadoopConf), fs, content, Option.empty());
- Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
- assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords());
- verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
-
- content = readHFileFromResources(complexHFile);
- verifyHFileReader(HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content),
- hfilePrefix, true, HFILE_COMPARATOR.getClass(), NUM_RECORDS_FIXTURE);
- hfileReader =
- new HoodieAvroHFileReader(hadoopConf, new Path(DUMMY_BASE_PATH), new CacheConfig(hadoopConf), fs, content, Option.empty());
- avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchemaWithUDT.avsc");
- assertEquals(NUM_RECORDS_FIXTURE, hfileReader.getTotalRecords());
- verifySimpleRecords(hfileReader.getRecordIterator(avroSchema));
-
- content = readHFileFromResources(bootstrapIndexFile);
- verifyHFileReader(HoodieHFileUtils.createHFileReader(fs, new Path(DUMMY_BASE_PATH), content),
- hfilePrefix, false, HFileBootstrapIndex.HoodieKVComparator.class, 4);
- }
-
- private Set getRandomKeys(int count, List keys) {
- Set rowKeys = new HashSet<>();
- int totalKeys = keys.size();
- while (rowKeys.size() < count) {
- int index = RANDOM.nextInt(totalKeys);
- if (!rowKeys.contains(index)) {
- rowKeys.add(keys.get(index));
- }
- }
- return rowKeys;
- }
-
- private byte[] readHFileFromResources(String filename) throws IOException {
- long size = TestHoodieHFileReaderWriter.class
- .getResource(filename).openConnection().getContentLength();
- return FileIOUtils.readAsByteArray(
- TestHoodieHFileReaderWriter.class.getResourceAsStream(filename), (int) size);
- }
-
- private void verifyHFileReader(
- HFile.Reader reader, String hfileName, boolean mayUseDefaultComparator,
- Class> clazz, int count) {
- // HFile version is 3
- assertEquals(3, reader.getTrailer().getMajorVersion());
- if (mayUseDefaultComparator && hfileName.contains("hudi_0_9")) {
- // Pre Hudi 0.10, the default comparator is used for metadata table HFiles
- // For bootstrap index HFiles, the custom comparator is always used
- assertEquals(CellComparatorImpl.class, reader.getComparator().getClass());
- } else {
- assertEquals(clazz, reader.getComparator().getClass());
- }
- assertEquals(count, reader.getEntries());
- }
-}
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersFactory.java
index 21c0e8108a53..d78b88306822 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersFactory.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/marker/TestWriteMarkersFactory.java
@@ -21,12 +21,12 @@
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.conf.Configuration;
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java
index 2196b6f0b630..a2949eb6eee1 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java
@@ -19,33 +19,43 @@
package org.apache.hudi.testutils;
import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieValidationException;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+import org.apache.hudi.io.storage.HoodieAvroHFileReaderImplBase;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
+import java.io.IOException;
import java.nio.file.Paths;
import java.util.Arrays;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
import static org.apache.hudi.common.model.HoodieRecord.COMMIT_SEQNO_METADATA_FIELD;
import static org.apache.hudi.common.model.HoodieRecord.COMMIT_TIME_METADATA_FIELD;
import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD;
import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD;
import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD;
+import static org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER;
import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.HOODIE_CONSUME_COMMIT;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -126,8 +136,22 @@ public static Map getRecordsMap(HoodieWriteConfig config,
.map(partitionPath -> Paths.get(config.getBasePath(), partitionPath).toString())
.collect(Collectors.toList());
return HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
- hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true).stream()
+ hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true).stream()
.collect(Collectors.toMap(rec -> rec.get(RECORD_KEY_METADATA_FIELD).toString(), Function.identity()));
}
+ public static Stream readHFile(Configuration conf, String[] paths) {
+ List valuesAsList = new LinkedList<>();
+ for (String path : paths) {
+ try (HoodieAvroHFileReaderImplBase reader = (HoodieAvroHFileReaderImplBase)
+ HoodieFileReaderFactory.getReaderFactory(HoodieRecord.HoodieRecordType.AVRO)
+ .getFileReader(DEFAULT_HUDI_CONFIG_FOR_READER, conf, new Path(path), HoodieFileFormat.HFILE)) {
+ valuesAsList.addAll(HoodieAvroHFileReaderImplBase.readAllRecords(reader)
+ .stream().map(e -> (GenericRecord) e).collect(Collectors.toList()));
+ } catch (IOException e) {
+ throw new HoodieException("Error reading HFile " + path, e);
+ }
+ }
+ return valuesAsList.stream();
+ }
}
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java
index 28173acd3aeb..bf7a3e33bf07 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java
@@ -27,7 +27,6 @@
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
-import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.RawTripTestPayload;
@@ -160,8 +159,7 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, HoodieIndex.
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
- .withRemoteServerPort(timelineServicePort)
- .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
+ .withRemoteServerPort(timelineServicePort).build());
if (StringUtils.nonEmpty(schemaStr)) {
builder.withSchema(schemaStr);
}
diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml
index 1c60b37194bc..96b2477236d2 100644
--- a/hudi-client/hudi-flink-client/pom.xml
+++ b/hudi-client/hudi-flink-client/pom.xml
@@ -20,12 +20,12 @@
hudi-client
org.apache.hudi
- 0.14.1
+ 0.15.0-SNAPSHOT
4.0.0
hudi-flink-client
- 0.14.1
+ 0.15.0-SNAPSHOT
hudi-flink-client
jar
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java
index 68c32acca24e..79bbeecaa56d 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java
@@ -51,11 +51,12 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.List;
import java.util.stream.Collectors;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
+
public class HoodieFlinkTableServiceClient extends BaseHoodieTableServiceClient>, List, List> {
private static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkTableServiceClient.class);
@@ -132,12 +133,12 @@ protected void completeClustering(
// commit to data table after committing to metadata table.
// We take the lock here to ensure all writes to metadata table happens within a single lock (single writer).
// Because more than one write to metadata table will result in conflicts since all of them updates the same partition.
- writeTableMetadata(table, clusteringCommitTime, metadata, writeStatuses.orElse(context.emptyHoodieData()));
+ writeTableMetadata(table, clusteringCommitTime, metadata, writeStatuses.orElseGet(context::emptyHoodieData));
LOG.info("Committing Clustering {} finished with result {}.", clusteringCommitTime, metadata);
table.getActiveTimeline().transitionReplaceInflightToComplete(
HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime),
- Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+ Option.of(getUTF8Bytes(metadata.toJsonString())));
} catch (IOException e) {
throw new HoodieClusteringException(
"Failed to commit " + table.getMetaClient().getBasePath() + " at time " + clusteringCommitTime, e);
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java
index 4a3109db60a3..a153ec15052d 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java
@@ -21,13 +21,13 @@
import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.hadoop.api.WriteSupport;
-import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.Map;
@@ -71,7 +71,7 @@ public HoodieBloomFilterRowDataWriteSupport(BloomFilter bloomFilter) {
@Override
protected byte[] getUTF8Bytes(String key) {
- return key.getBytes(StandardCharsets.UTF_8);
+ return StringUtils.getUTF8Bytes(key);
}
}
}
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java
index 5f9b71d4c9fc..3dca687e9e85 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java
@@ -46,7 +46,6 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.Collections;
import java.util.Iterator;
@@ -55,6 +54,8 @@
import java.util.Map;
import java.util.stream.Collectors;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
+
/**
* With {@code org.apache.hudi.operator.partitioner.BucketAssigner}, each hoodie record
* is tagged with a bucket ID (partition path + fileID) in streaming way. All the records consumed by this
@@ -156,7 +157,7 @@ protected void commit(Option