[HUDI-159] Redesigning bundles for lighter-weight integrations
- Documented principles applied for redesign at packaging/README.md - No longer depends on incl commons-codec, commons-io, commons-pool, commons-dbcp, commons-lang, commons-logging, avro-mapred - Introduce new FileIOUtils & added checkstyle rule for illegal import of above - Parquet, Avro dependencies moved to provided scope to enable being picked up from Hive/Spark/Presto instead - Pickup jackson jars for Hive sync tool from HIVE_HOME & unbundling jackson everywhere - Remove hive-jdbc standalone jar from being bundled in Spark/Hive/Utilities bundles - 6.5x reduced number of classes across bundles
This commit is contained in:
committed by
Balaji Varadarajan
parent
0e6f078ec4
commit
7a973a6944
@@ -18,13 +18,14 @@
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Calendar;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.time.DateUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -45,8 +46,8 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// The earliest partition to compact - current day minus the target partitions limit
|
||||
String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
|
||||
.getTargetPartitionsPerDayBasedCompaction()));
|
||||
String earliestPartitionPathToCompact = dateFormat.format(
|
||||
getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
|
||||
// Filter out all partitions greater than earliestPartitionPathToCompact
|
||||
List<HoodieCompactionOperation> eligibleCompactionOperations = operations.stream()
|
||||
.collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
|
||||
@@ -61,8 +62,8 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
|
||||
@Override
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
|
||||
// The earliest partition to compact - current day minus the target partitions limit
|
||||
String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
|
||||
.getTargetPartitionsPerDayBasedCompaction()));
|
||||
String earliestPartitionPathToCompact = dateFormat.format(
|
||||
getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
|
||||
// Get all partitions and sort them
|
||||
List<String> filteredPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
|
||||
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
|
||||
@@ -70,4 +71,11 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
|
||||
.collect(Collectors.toList());
|
||||
return filteredPartitionPaths;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static Date getDateAtOffsetFromToday(int offset) {
|
||||
Calendar calendar = Calendar.getInstance();
|
||||
calendar.add(Calendar.DATE, offset);
|
||||
return calendar.getTime();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,7 +22,6 @@ import com.codahale.metrics.Gauge;
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import com.google.common.io.Closeables;
|
||||
import java.io.Closeable;
|
||||
import org.apache.commons.configuration.ConfigurationException;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -37,9 +36,9 @@ public class Metrics {
|
||||
private static volatile boolean initialized = false;
|
||||
private static Metrics metrics = null;
|
||||
private final MetricRegistry registry;
|
||||
private MetricsReporter reporter = null;
|
||||
private MetricsReporter reporter;
|
||||
|
||||
private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
|
||||
private Metrics(HoodieWriteConfig metricConfig) {
|
||||
registry = new MetricRegistry();
|
||||
|
||||
reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
|
||||
@@ -72,7 +71,7 @@ public class Metrics {
|
||||
}
|
||||
try {
|
||||
metrics = new Metrics(metricConfig);
|
||||
} catch (ConfigurationException e) {
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
initialized = true;
|
||||
|
||||
@@ -38,7 +38,6 @@ import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
@@ -55,6 +54,7 @@ import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -562,9 +562,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = IOUtils.toString(inputStream, "UTF-8");
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
|
||||
HoodieCommitMetadata.class);
|
||||
String everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything, HoodieCommitMetadata.class);
|
||||
HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
|
||||
inputStream.close();
|
||||
|
||||
@@ -600,7 +599,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
// Read from commit file
|
||||
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
String everything = IOUtils.toString(inputStream, "UTF-8");
|
||||
String everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
|
||||
HoodieCommitMetadata.class);
|
||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
|
||||
@@ -629,7 +628,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
// Read from commit file
|
||||
filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
|
||||
inputStream = new FileInputStream(filename);
|
||||
everything = IOUtils.toString(inputStream, "UTF-8");
|
||||
everything = FileIOUtils.readAsUTFString(inputStream);
|
||||
metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), HoodieCommitMetadata.class);
|
||||
rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
|
||||
|
||||
@@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -32,11 +31,11 @@ import java.util.zip.DeflaterOutputStream;
|
||||
import java.util.zip.InflaterInputStream;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.MercifulJsonConverter;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
@@ -132,10 +131,9 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
|
||||
|
||||
|
||||
private String unCompressData(byte[] data) throws IOException {
|
||||
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
|
||||
StringWriter sw = new StringWriter(dataSize);
|
||||
IOUtils.copy(iis, sw);
|
||||
return sw.toString();
|
||||
try (InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data))) {
|
||||
return FileIOUtils.readAsUTFString(iis, dataSize);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -36,11 +36,11 @@ import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SizeEstimator;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
|
||||
@@ -82,7 +82,7 @@ public class TestBoundedInMemoryQueue {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
// Produce
|
||||
Future<Boolean> resFuture =
|
||||
executorService.submit(() -> {
|
||||
@@ -122,7 +122,7 @@ public class TestBoundedInMemoryQueue {
|
||||
final List<List<HoodieRecord>> recs = new ArrayList<>();
|
||||
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
|
||||
|
||||
// Record Key to <Producer Index, Rec Index within a producer>
|
||||
Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
|
||||
|
||||
@@ -27,7 +27,6 @@ import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
@@ -39,6 +38,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
@@ -151,7 +151,7 @@ public class TestUpdateMapFunction implements Serializable {
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream(schema));
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -37,7 +37,6 @@ import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
@@ -48,6 +47,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -103,7 +103,7 @@ public class TestHoodieBloomIndex {
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
// We have some records to be tagged (two different partitions)
|
||||
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
}
|
||||
|
||||
|
||||
@@ -34,7 +34,6 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.HoodieClientTestUtils;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
@@ -44,6 +43,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -78,7 +78,7 @@ public class TestHoodieGlobalBloomIndex {
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
|
||||
// We have some records to be tagged (two different partitions)
|
||||
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
}
|
||||
|
||||
|
||||
@@ -31,7 +31,6 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.time.DateUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
@@ -161,11 +160,12 @@ public class TestHoodieCompactionStrategy {
|
||||
SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
|
||||
Date today = new Date();
|
||||
String currentDay = format.format(today);
|
||||
String currentDayMinus1 = format.format(DateUtils.addDays(today, -1));
|
||||
String currentDayMinus2 = format.format(DateUtils.addDays(today, -2));
|
||||
String currentDayMinus3 = format.format(DateUtils.addDays(today, -3));
|
||||
String currentDayPlus1 = format.format(DateUtils.addDays(today, 1));
|
||||
String currentDayPlus5 = format.format(DateUtils.addDays(today, 5));
|
||||
|
||||
String currentDayMinus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-1));
|
||||
String currentDayMinus2 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-2));
|
||||
String currentDayMinus3 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-3));
|
||||
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
|
||||
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder()
|
||||
.put(120 * MB, currentDay)
|
||||
@@ -208,11 +208,12 @@ public class TestHoodieCompactionStrategy {
|
||||
SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
|
||||
Date today = new Date();
|
||||
String currentDay = format.format(today);
|
||||
String currentDayMinus1 = format.format(DateUtils.addDays(today, -1));
|
||||
String currentDayMinus2 = format.format(DateUtils.addDays(today, -2));
|
||||
String currentDayMinus3 = format.format(DateUtils.addDays(today, -3));
|
||||
String currentDayPlus1 = format.format(DateUtils.addDays(today, 1));
|
||||
String currentDayPlus5 = format.format(DateUtils.addDays(today, 5));
|
||||
|
||||
String currentDayMinus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-1));
|
||||
String currentDayMinus2 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-2));
|
||||
String currentDayMinus3 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-3));
|
||||
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
|
||||
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder()
|
||||
.put(120 * MB, currentDay)
|
||||
|
||||
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import org.apache.commons.configuration.ConfigurationException;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
@@ -33,7 +32,7 @@ public class TestHoodieMetrics {
|
||||
private HoodieMetrics metrics = null;
|
||||
|
||||
@Before
|
||||
public void start() throws ConfigurationException {
|
||||
public void start() {
|
||||
HoodieWriteConfig config = mock(HoodieWriteConfig.class);
|
||||
when(config.isMetricsOn()).thenReturn(true);
|
||||
when(config.getMetricsReporterType()).thenReturn(MetricsReporterType.INMEMORY);
|
||||
|
||||
@@ -30,7 +30,6 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
@@ -45,6 +44,7 @@ import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -117,7 +117,7 @@ public class TestCopyOnWriteTable {
|
||||
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user