Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
63 commits
Select commit Hold shift + click to select a range
bd8e50c
Adding parquet data block and inline read support
nsivabalan Nov 5, 2021
6190789
Assert accepted `Path` instances actually represent InlineFS block pa…
Dec 14, 2021
3bdba30
Fixing compilation
Dec 14, 2021
8f720a9
Tidying up
Dec 14, 2021
fe23d29
Cleaned up `HoodieDataBlock`:
Dec 15, 2021
0896844
Tidying up
Dec 15, 2021
2caa71f
Fixing layout
Dec 15, 2021
3bf54eb
Fixing `HoodieHFileDataBlock` to properly handle cases when point loo…
Dec 15, 2021
8666008
Tidying up:
Dec 15, 2021
06a1dcb
Control ser/de flow fully from w/in the `HodieDataBlock`
Dec 15, 2021
c56a5bd
Added assertions for `HoodieDataBlock`;
Dec 15, 2021
811aabb
Tidying up
Dec 15, 2021
f82f136
Added `ByteBufferBackedInputStream`
Dec 15, 2021
dabd871
Rebased `HoodiHFileReader` onto `ByteBufferBackedInputStream` (to av…
Dec 15, 2021
f715050
Completed `HoodieParquetDataBlock`:
Dec 15, 2021
77993e2
Tidying up
Dec 16, 2021
c6534d7
Cleaned up `HoodieLogFileReader`;
Dec 16, 2021
5d1aa54
Missing license
Dec 16, 2021
2ec97cb
Tidying up;
Dec 16, 2021
59b1911
Reverted `hasContentLength` change
Dec 16, 2021
1839701
Fixed NPEs
Dec 16, 2021
ee17c94
Tidied up config
Dec 16, 2021
e15d770
Added tests for Parquet data blocks
Dec 16, 2021
bfb5bf7
Streamlining
Dec 16, 2021
5935a41
Extracted `ByteBufferBackedInputFile` as a standalone class
Dec 16, 2021
e353851
Extracted `OutputStreamBackedOutputFile` as a standalone class;
Dec 16, 2021
5183297
Missing license
Dec 16, 2021
1c533d9
Make sure `LogBlock`s schema is always set to either Reader's schema …
Dec 16, 2021
1ce9b6e
Added `Option.or`
Dec 16, 2021
12bb746
Abstracted common point-wise record lookups w/in `HoodieDataBlock` to…
Dec 16, 2021
c30f2e7
Disable point-lookups for Parquet data blocks
Dec 16, 2021
f0c9597
Tidying up
Dec 16, 2021
14dbeb8
Made `readRecordsFromContent` overridable to make sure that Parquet B…
Dec 17, 2021
1df7fb7
Tidying up
Dec 17, 2021
6ad6ac8
Repacked `HoodieLogFile` w/in `HoodieLogFileReader` to make sure that…
Dec 20, 2021
7fd5105
Down-scoping some utils
Dec 20, 2021
3934a42
Added test to assert that Data Block produces records w/ correct proj…
Dec 20, 2021
1f2a980
Tydying up
Dec 20, 2021
2b7ac1c
Make sure `HoodieAvroDataBlock` doesn't alter original records' list
Dec 20, 2021
c19e181
`lint`
Dec 20, 2021
fa5063d
Fixing styling
Dec 22, 2021
bc73ece
Tidying up
Jan 5, 2022
4fc9fbf
Cleaned up ctors used only in tests
Jan 5, 2022
1d65847
Thread through appropritately configured Parquet compression codec in…
Jan 5, 2022
3f286ca
Thread through appropriately configured Parquet compression codec int…
Jan 5, 2022
a232513
Thread t/h Hadoop `Configuration` to the Log Blocks to make sure prop…
Jan 5, 2022
6594fc2
Properly respect Hadoop Configuration
Jan 5, 2022
79accfa
Fixed compilation
Jan 5, 2022
f5df8bd
Leverage appropriately configured HFile compression algo
Jan 5, 2022
0cef98e
`lint`
Jan 5, 2022
640828b
Avoid superfluous `fs.getFileStatus` by amending the Path w/ Scheme d…
Jan 5, 2022
a4509d1
Added UT for `ByteBufferBackedInputStream`
Jan 5, 2022
a08d0ce
Fixed how Path is qualified w/ FS default URI, scheme
Jan 5, 2022
98f44f5
Properly pass in file-size when re-packaging `HoodieLogFile`;
Jan 5, 2022
12d0ed8
Tidying up
Jan 14, 2022
d451371
Moved `LOGFILE_DATA_BLOCK_FORMAT` into `HoodieWriteConfig`
Jan 14, 2022
ae505bc
Tidying up
Jan 14, 2022
c455ea0
Unused imports
Jan 14, 2022
c47b840
After rebase fixes
Feb 1, 2022
8743a38
Tidying up
Feb 1, 2022
10221e0
Tidying up more
Feb 1, 2022
d0b17e5
`lint`
Feb 1, 2022
29dd840
Fixed writers' ctor of the `HoodieHFileDataBlock`
Feb 2, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ public void init() throws IOException, InterruptedException, URISyntaxException
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
dataBlock = new HoodieAvroDataBlock(records, header);
dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
writer.appendBlock(dataBlock);
}
}
Expand Down Expand Up @@ -188,7 +188,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header, HoodieRecord.RECORD_KEY_METADATA_FIELD);
writer.appendBlock(dataBlock);
} finally {
if (writer != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,14 +83,17 @@ public class HoodieStorageConfig extends HoodieConfig {
.withDocumentation("Lower values increase the size of metadata tracked within HFile, but can offer potentially "
+ "faster lookup times.");

// used to size log files
public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_FORMAT = ConfigProperty
.key("hoodie.logfile.data.block.format")
.noDefaultValue()
.withDocumentation("Format of the data block within delta logs. Following formats are currently supported \"avro\", \"hfile\", \"parquet\"");

public static final ConfigProperty<String> LOGFILE_MAX_SIZE = ConfigProperty
.key("hoodie.logfile.max.size")
.defaultValue(String.valueOf(1024 * 1024 * 1024)) // 1 GB
.withDocumentation("LogFile max size. This is the maximum size allowed for a log file "
+ "before it is rolled over to the next version.");

// used to size data blocks in log file
public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_MAX_SIZE = ConfigProperty
.key("hoodie.logfile.data.block.max.size")
.defaultValue(String.valueOf(256 * 1024 * 1024))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,11 @@
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.model.WriteConcurrencyMode;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.marker.MarkerType;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.metrics.HoodieMetricsConfig;
Expand Down Expand Up @@ -1506,6 +1508,11 @@ public String parquetOutputTimestampType() {
return getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE);
}

public Option<HoodieLogBlock.HoodieLogBlockType> getLogDataBlockFormat() {
return Option.ofNullable(getString(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT))
.map(HoodieLogBlock.HoodieLogBlockType::fromId);
}

public long getLogFileMaxSize() {
return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,17 +38,20 @@
import org.apache.hudi.common.table.log.AppendResult;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.SizeEstimator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieAppendException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.table.HoodieTable;

Expand Down Expand Up @@ -360,13 +363,13 @@ protected void appendDataAndDeleteBlocks(Map<HeaderMetadataType, String> header)
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchemaWithMetaFields.toString());
List<HoodieLogBlock> blocks = new ArrayList<>(2);
if (recordList.size() > 0) {
if (config.populateMetaFields()) {
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
} else {
final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header, keyField));
}
String keyField = config.populateMetaFields()
? HoodieRecord.RECORD_KEY_METADATA_FIELD
: hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();

blocks.add(getBlock(config, pickLogDataBlockFormat(), recordList, header, keyField));
}

if (keysToDelete.size() > 0) {
blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header));
}
Expand Down Expand Up @@ -497,4 +500,40 @@ private void flushToDiskIfRequired(HoodieRecord record) {
numberOfRecords = 0;
}
}

private HoodieLogBlock.HoodieLogBlockType pickLogDataBlockFormat() {
Option<HoodieLogBlock.HoodieLogBlockType> logBlockTypeOpt = config.getLogDataBlockFormat();
if (logBlockTypeOpt.isPresent()) {
return logBlockTypeOpt.get();
}

// Fallback to deduce data-block type based on the base file format
switch (hoodieTable.getBaseFileFormat()) {
case PARQUET:
case ORC:
return HoodieLogBlock.HoodieLogBlockType.AVRO_DATA_BLOCK;
case HFILE:
return HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK;
default:
throw new HoodieException("Base file format " + hoodieTable.getBaseFileFormat()
+ " does not have associated log block type");
}
}

private static HoodieLogBlock getBlock(HoodieWriteConfig writeConfig,
HoodieLogBlock.HoodieLogBlockType logDataBlockFormat,
List<IndexedRecord> recordList,
Map<HeaderMetadataType, String> header,
String keyField) {
switch (logDataBlockFormat) {
case AVRO_DATA_BLOCK:
return new HoodieAvroDataBlock(recordList, header, keyField);
case HFILE_DATA_BLOCK:
return new HoodieHFileDataBlock(recordList, header, writeConfig.getHFileCompressionAlgorithm());
case PARQUET_DATA_BLOCK:
return new HoodieParquetDataBlock(recordList, header, keyField, writeConfig.getParquetCompressionCodec());
default:
throw new HoodieException("Data block format " + logDataBlockFormat + " not implemented");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,13 +51,23 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final TaskContextSupplier taskContextSupplier;
private final boolean populateMetaFields;

public HoodieParquetWriter(String instantTime, Path file, HoodieAvroParquetConfig parquetConfig,
Schema schema, TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException {
public HoodieParquetWriter(String instantTime,
Path file,
HoodieAvroParquetConfig parquetConfig,
Schema schema,
TaskContextSupplier taskContextSupplier,
boolean populateMetaFields) throws IOException {
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
parquetConfig.dictionaryEnabled(), DEFAULT_IS_VALIDATING_ENABLED,
DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
ParquetFileWriter.Mode.CREATE,
parquetConfig.getWriteSupport(),
parquetConfig.getCompressionCodecName(),
parquetConfig.getBlockSize(),
parquetConfig.getPageSize(),
parquetConfig.getPageSize(),
parquetConfig.dictionaryEnabled(),
DEFAULT_IS_VALIDATING_ENABLED,
DEFAULT_WRITER_VERSION,
FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs =
(HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,11 @@

package org.apache.hudi.table;

import org.apache.avro.Schema;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
Expand All @@ -44,7 +48,6 @@
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
Expand All @@ -69,11 +72,6 @@
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
import org.apache.hudi.table.marker.WriteMarkers;
import org.apache.hudi.table.marker.WriteMarkersFactory;

import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.storage.HoodieLayoutFactory;
import org.apache.hudi.table.storage.HoodieStorageLayout;
import org.apache.log4j.LogManager;
Expand Down Expand Up @@ -719,19 +717,6 @@ public HoodieFileFormat getLogFileFormat() {
return metaClient.getTableConfig().getLogFileFormat();
}

public HoodieLogBlockType getLogDataBlockFormat() {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is a very good cleanup. thanks!

switch (getBaseFileFormat()) {
case PARQUET:
case ORC:
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 String getBaseFileExtension() {
return getBaseFileFormat().getFileExtension();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ private Pair<String, HoodieLogFile> appendRecordsToLogFile(List<HoodieRecord> gr
LOG.warn("Failed to convert record " + r.toString(), e);
return null;
}
}).collect(Collectors.toList()), header));
}).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD));
return Pair.of(partitionPath, logWriter.getLogFile());
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.apache.hudi.testutils;

import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
Expand All @@ -39,6 +40,7 @@
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.HashMap;
Expand Down Expand Up @@ -132,12 +134,12 @@ private Pair<String, HoodieLogFile> appendRecordsToLogFile(List<HoodieRecord> gr
try {
GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get();
HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), "");
return (org.apache.avro.generic.IndexedRecord) val;
} catch (java.io.IOException e) {
return (IndexedRecord) val;
} catch (IOException e) {
LOG.warn("Failed to convert record " + r.toString(), e);
return null;
}
}).collect(Collectors.toList()), header));
}).collect(Collectors.toList()), header, HoodieRecord.RECORD_KEY_METADATA_FIELD));
return Pair.of(partitionPath, logWriter.getLogFile());
}
}
Expand Down
65 changes: 21 additions & 44 deletions hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,6 @@

package org.apache.hudi.avro;

import org.apache.hudi.common.config.SerializableSchema;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.SchemaCompatibilityException;

import org.apache.avro.Conversions.DecimalConversion;
import org.apache.avro.JsonProperties;
import org.apache.avro.LogicalTypes;
Expand All @@ -50,15 +39,22 @@
import org.apache.avro.io.JsonDecoder;
import org.apache.avro.io.JsonEncoder;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hudi.common.config.SerializableSchema;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.SchemaCompatibilityException;

import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Timestamp;
import java.time.LocalDate;
import java.util.ArrayList;
Expand All @@ -67,8 +63,6 @@
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;

/**
* Helper class to do common stuff across Avro.
Expand Down Expand Up @@ -343,7 +337,7 @@ public static GenericRecord stitchRecords(GenericRecord left, GenericRecord righ
}

/**
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new
* Given an Avro record with a given schema, rewrites it into the new schema while setting fields only from the new
* schema.
* NOTE: Here, the assumption is that you cannot go from an evolved schema (schema with (N) fields)
* to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the
Expand Down Expand Up @@ -377,6 +371,16 @@ public static GenericRecord rewriteRecord(GenericRecord oldRecord, Schema newSch
return newRecord;
}

/**
* Converts list of {@link GenericRecord} provided into the {@link GenericRecord} adhering to the
* provided {@code newSchema}.
*
* To better understand conversion rules please check {@link #rewriteRecord(GenericRecord, Schema)}
*/
public static List<GenericRecord> rewriteRecords(List<GenericRecord> records, Schema newSchema) {
return records.stream().map(r -> rewriteRecord(r, newSchema)).collect(Collectors.toList());
}

private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field f) {
// cache the result of oldRecord.get() to save CPU expensive hash lookup
Schema oldSchema = oldRecord.getSchema();
Expand All @@ -392,33 +396,6 @@ private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRec
}
}

public static byte[] compress(String text) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
OutputStream out = new DeflaterOutputStream(baos);
out.write(text.getBytes(StandardCharsets.UTF_8));
out.close();
} catch (IOException e) {
throw new HoodieIOException("IOException while compressing text " + text, e);
}
return baos.toByteArray();
}

public static String decompress(byte[] bytes) {
InputStream in = new InflaterInputStream(new ByteArrayInputStream(bytes));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
byte[] buffer = new byte[8192];
int len;
while ((len = in.read(buffer)) > 0) {
baos.write(buffer, 0, len);
}
return new String(baos.toByteArray(), StandardCharsets.UTF_8);
} catch (IOException e) {
throw new HoodieIOException("IOException while decompressing text", e);
}
}

/**
* Generate a reader schema off the provided writeSchema, to just project out the provided columns.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,10 @@

package org.apache.hudi.avro;

import org.apache.avro.Schema;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
import org.apache.hudi.common.util.Option;

import org.apache.avro.Schema;
import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.schema.MessageType;
Expand Down
Loading