bvaradar commented on a change in pull request #1687:
URL: https://github.com/apache/hudi/pull/1687#discussion_r440454058
##########
File path: hudi-client/pom.xml
##########
@@ -185,6 +185,12 @@
<artifactId>hbase-client</artifactId>
<version>${hbase.version}</version>
</dependency>
+ <dependency>
Review comment:
Is this only needed on the write side ?
##########
File path:
hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
##########
@@ -217,6 +239,51 @@ public static SparkConf getSparkConfForTest(String
appName) {
}
}
+ public static Dataset<Row> readHFile(JavaSparkContext jsc, SQLContext
sqlContext, String[] paths) {
+ // TODO: this should be ported to use HoodieStorageReader
+ List<byte[]> valuesAsList = new LinkedList<>();
+
+ FileSystem fs = FSUtils.getFs(paths[0], jsc.hadoopConfiguration());
+ CacheConfig cacheConfig = new CacheConfig(fs.getConf());
+ Schema schema = null;
+ for (String path : paths) {
Review comment:
We might eventually need such support when supporting spark datasource
support on Hoodie HFile table. It would be better to move this logic to a Utils
class to reuse.
##########
File path:
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.common.table.log.block;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.FSDataInputStream;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * DataBlock contains a list of records serialized using formats compatible
with the base file format.
+ * For each base file format there is a corresponding DataBlock format.
+ *
+ * The Datablock contains:
Review comment:
Can you confirm there is no change to order and kinds of fields stored
here. I want to make sure if we can read log files written by 0.5.x using this
change.
##########
File path:
hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
##########
@@ -34,29 +34,28 @@
import java.io.IOException;
-import static org.apache.hudi.common.model.HoodieFileFormat.HOODIE_LOG;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
+import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
-public class HoodieStorageWriterFactory {
+public class HoodieFileWriterFactory {
- public static <T extends HoodieRecordPayload, R extends IndexedRecord>
HoodieStorageWriter<R> getStorageWriter(
+ public static <T extends HoodieRecordPayload, R extends IndexedRecord>
HoodieFileWriter<R> getFileWriter(
Review comment:
We have a family of things that needs to be constructed:
1. StorageWriter
2. StorageReader
3. MergeAlgorithm
In this case, AbstractFactoryPattern would be useful
https://en.wikipedia.org/wiki/Abstract_factory_pattern#/media/File:Abstract_factory_UML.svg
pattern would be ideal for this case.
This is more or less similar to what has been done here. Currently, there is
a separate Writer and Reader Factory. What do you think about creating an
AbstractFactory and have one concrete implementation for Parquet and another
for HFile where all the above objects are constructed ?
##########
File path:
hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.io.Writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * HoodieHFileWriter writes IndexedRecords into an HFile. The record's key is
used as the key and the
+ * AVRO encoded record bytes are saved as the value.
+ *
+ * Limitations (compared to columnar formats like Parquet or ORC):
+ * 1. Records should be added in order of keys
+ * 2. There are no column stats
+ */
+public class HoodieHFileWriter<T extends HoodieRecordPayload, R extends
IndexedRecord>
+ implements HoodieFileWriter<R> {
+ private static AtomicLong recordIndex = new AtomicLong(1);
+ private static final Logger LOG =
LogManager.getLogger(HoodieHFileWriter.class);
+
+ public static final String KEY_SCHEMA = "schema";
+ public static final String KEY_BLOOM_FILTER_META_BLOCK = "bloomFilter";
+ public static final String KEY_BLOOM_FILTER_TYPE_CODE =
"bloomFilterTypeCode";
+ public static final String KEY_MIN_RECORD = "minRecordKey";
+ public static final String KEY_MAX_RECORD = "maxRecordKey";
+
+ private final Path file;
+ private HoodieHFileConfig hfileConfig;
+ private final HoodieWrapperFileSystem fs;
+ private final long maxFileSize;
+ private final String instantTime;
+ private final SparkTaskContextSupplier sparkTaskContextSupplier;
+ private HFile.Writer writer;
+ private String minRecordKey;
+ private String maxRecordKey;
+
+ public HoodieHFileWriter(String instantTime, Path file, HoodieHFileConfig
hfileConfig, Schema schema,
+ SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
+
+ Configuration conf = registerFileSystem(file, hfileConfig.getHadoopConf());
+ this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, conf);
+ this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(conf);
+ this.hfileConfig = hfileConfig;
+
+ // We cannot accurately measure the snappy compressed output file size. We
are choosing a
+ // conservative 10%
+ // TODO - compute this compression ratio dynamically by looking at the
bytes written to the
+ // stream and the actual file size reported by HDFS
+ // this.maxFileSize = hfileConfig.getMaxFileSize()
+ // + Math.round(hfileConfig.getMaxFileSize() *
hfileConfig.getCompressionRatio());
+ this.maxFileSize = hfileConfig.getMaxFileSize();
+ this.instantTime = instantTime;
+ this.sparkTaskContextSupplier = sparkTaskContextSupplier;
+
+ HFileContext context = new
HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
+ .withCompression(hfileConfig.getCompressionAlgorithm())
+ .build();
+ CacheConfig cacheConfig = new CacheConfig(conf);
+ this.writer = HFile.getWriterFactory(conf, cacheConfig).withPath(this.fs,
this.file).withFileContext(context).create();
+
+ writer.appendFileInfo(KEY_SCHEMA.getBytes(), schema.toString().getBytes());
+ }
+
+ public static Configuration registerFileSystem(Path file, Configuration
conf) {
Review comment:
Looks like this is duplicated. Can you refactor to reuse this code.
##########
File path: hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
##########
@@ -133,6 +136,12 @@ private void init(String fileId, String partitionPath,
HoodieBaseFile dataFileTo
// Create the writer for writing the new version file
storageWriter =
HoodieStorageWriterFactory.getStorageWriter(instantTime,
newFilePath, hoodieTable, config, writerSchema, sparkTaskContextSupplier);
+
+ if (hoodieTable.requireSortedRecords()) {
Review comment:
Can we have HashBasedJoin and SortMergeJoin as first level abstractions
which both taken in 2 streams of records to be merged ? We can then plugin the
algorithm depending on storage preference.
##########
File path:
hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
##########
@@ -34,29 +34,28 @@
import java.io.IOException;
-import static org.apache.hudi.common.model.HoodieFileFormat.HOODIE_LOG;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
+import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
-public class HoodieStorageWriterFactory {
+public class HoodieFileWriterFactory {
- public static <T extends HoodieRecordPayload, R extends IndexedRecord>
HoodieStorageWriter<R> getStorageWriter(
+ public static <T extends HoodieRecordPayload, R extends IndexedRecord>
HoodieFileWriter<R> getFileWriter(
String instantTime, Path path, HoodieTable<T> hoodieTable,
HoodieWriteConfig config, Schema schema,
SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
- final String name = path.getName();
- final String extension = FSUtils.isLogFile(path) ?
HOODIE_LOG.getFileExtension() : FSUtils.getFileExtension(name);
+ final String extension = FSUtils.getFileExtension(path.getName());
if (PARQUET.getFileExtension().equals(extension)) {
Review comment:
Please see above for AbstractFactory suggestion.
##########
File path: hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
##########
@@ -562,4 +566,28 @@ public void validateInsertSchema() throws
HoodieInsertException {
throw new HoodieInsertException("Failed insert schema compability
check.", e);
}
}
+
+ public HoodieFileFormat getBaseFileFormat() {
+ return metaClient.getTableConfig().getBaseFileFormat();
+ }
+
+ public HoodieFileFormat getLogFileFormat() {
+ return metaClient.getTableConfig().getLogFileFormat();
+ }
+
+ public HoodieLogBlockType getLogDataBlockFormat() {
+ switch (getBaseFileFormat()) {
+ case PARQUET:
+ return HoodieLogBlockType.AVRO_DATA_BLOCK;
+ case HFILE:
+ return HoodieLogBlockType.HFILE_DATA_BLOCK;
+ default:
+ throw new HoodieException("Base file format " + getBaseFileFormat()
+ + " does not have associated log block format");
+ }
+ }
+
+ public boolean requireSortedRecords() {
Review comment:
Instead of this, lets return merge algorithm to be employed.
##########
File path:
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.hadoop;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * HoodieInputFormat for HUDI datasets which store data in HFile base file
format.
+ */
+@UseFileSplitsFromInputFormat
+public class HoodieHFileInputFormat extends FileInputFormat<NullWritable,
ArrayWritable> implements Configurable {
Review comment:
Can we add tons of tests (expand on something like
TestHoodieParquetInputFormat) for HFIle. Also, I suspect when you put things
end to end like run a proper hive/presto query, you will run into package
version mismatches which is what I had to deal with when doing metadata
bootstrap.
##########
File path:
hudi-client/src/main/java/org/apache/hudi/table/action/commit/WriteHelper.java
##########
@@ -52,6 +52,10 @@
}
Duration indexLookupDuration = Duration.between(lookupBegin,
Instant.now());
+ if (table.requireSortedRecords()) {
+ taggedRecords = taggedRecords.sortBy(r -> r.getRecordKey(), true,
taggedRecords.getNumPartitions());
Review comment:
This is one another shuffle which will cause performance overhead. We
dont need universal order. We only need to order records that are getting
written to a single partition.
we should just do
https://spark.apache.org/docs/1.2.0/api/java/org/apache/spark/rdd/OrderedRDDFunctions.html#repartitionAndSortWithinPartitions(org.apache.spark.Partitioner)
when partitioning (See BaseCommitExecutor.java)
##########
File path: hudi-common/pom.xml
##########
@@ -78,6 +92,38 @@
</imports>
</configuration>
</plugin>
+ <plugin>
Review comment:
+1
##########
File path:
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.common.table.log.block;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.FSDataInputStream;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * DataBlock contains a list of records serialized using formats compatible
with the base file format.
+ * For each base file format there is a corresponding DataBlock format.
+ *
+ * The Datablock contains:
+ * 1. Data Block version
+ * 2. Total number of records in the block
+ * 3. Size of a record
Review comment:
Average Size ?
##########
File path:
hudi-common/src/main/scala/com/databricks/spark/avro/HoodieAvroSchemaConversion.scala
##########
@@ -0,0 +1,31 @@
+/*
+ * 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 com.databricks.spark.avro
+
+import org.apache.avro.Schema
+import org.apache.avro.generic.GenericRecord
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.types.DataType
+
+/**
+ * This helper class is required since SchemaConverters.createConverterToSQL
is currently private.
+ */
+object HoodieAvroSchemaConversion {
+ def createConverterToSQL(avroSchema: Schema, sparkSchema: DataType):
(GenericRecord) => Row =
Review comment:
+1
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]