Timeline Service with Incremental View Syncing support
This commit is contained in:
committed by
vinoth chandar
parent
446f99aa0f
commit
64fec64097
@@ -66,7 +66,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
|
||||
@@ -143,7 +143,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testRollbackInflightIngestionWithPendingCompaction() throws Exception {
|
||||
// Rollback inflight ingestion when there is pending compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
@@ -195,7 +195,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testInflightCompaction() throws Exception {
|
||||
// There is inflight compaction. Subsequent compaction run must work correctly
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
@@ -211,7 +211,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
// Schedule and mark compaction instant as inflight
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
|
||||
|
||||
@@ -227,7 +227,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testScheduleIngestionBeforePendingCompaction() throws Exception {
|
||||
// Case: Failure case. Latest pending compaction instant time must be earlier than this instant time
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
@@ -264,7 +264,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
@@ -301,7 +301,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
@@ -351,7 +351,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testCompactionAfterTwoDeltaCommits() throws Exception {
|
||||
// No Delta Commits after compaction request
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
@@ -363,7 +363,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false);
|
||||
}
|
||||
|
||||
@@ -371,7 +371,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testInterleavedCompaction() throws Exception {
|
||||
//Case: Two delta commits before and after compaction schedule
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
@@ -386,7 +386,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
records, cfg, true, new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
|
||||
@@ -402,12 +402,11 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
final Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation,
|
||||
HoodieWriteConfig cfg) throws IOException {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable table = getHoodieTable(metaClient, cfg);
|
||||
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table, cfg);
|
||||
fileSliceList.forEach(fileSlice -> {
|
||||
Pair<String, HoodieCompactionOperation> opPair = fgIdToCompactionOperation.get(fileSlice.getFileGroupId());
|
||||
if (opPair != null) {
|
||||
System.out.println("FileSlice :" + fileSlice);
|
||||
assertTrue("Expect baseInstant to match compaction Instant",
|
||||
fileSlice.getBaseInstantTime().equals(opPair.getKey()));
|
||||
assertTrue("Expect atleast one log file to be present where the latest delta commit was written",
|
||||
@@ -448,7 +447,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
}
|
||||
TestHoodieClientBase.assertNoWriteErrors(statusList);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
List<HoodieDataFile> dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg);
|
||||
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
|
||||
dataFilesToRead.stream().findAny().isPresent());
|
||||
@@ -515,8 +514,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
}
|
||||
|
||||
// verify that there is a commit
|
||||
table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), cfg, jsc);
|
||||
table = getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), cfg);
|
||||
HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants();
|
||||
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
|
||||
assertEquals("Expect compaction instant time to be the latest commit time",
|
||||
@@ -571,4 +570,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
|
||||
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
return HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import org.junit.After;
|
||||
|
||||
public class TestAsyncCompactionWithEmbeddedTimelineServer extends TestAsyncCompaction {
|
||||
|
||||
@Override
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
HoodieWriteConfig.Builder builder = super.getConfigBuilder(autoCommit);
|
||||
try {
|
||||
return builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
((SyncableFileSystemView) (table.getRTFileSystemView())).reset();
|
||||
return table;
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
}
|
||||
@@ -124,7 +124,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, client.config, jsc);
|
||||
|
||||
assertFalse(table.getCompletedCommitsTimeline().empty());
|
||||
String commitTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp();
|
||||
@@ -194,7 +194,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.withParallelism(1, 1).withBulkInsertParallelism(1)
|
||||
.withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true)
|
||||
.build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
|
||||
@@ -355,7 +355,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainCommits(maxCommits).build())
|
||||
.withParallelism(1, 1).withBulkInsertParallelism(1)
|
||||
.withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
|
||||
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.IOException;
|
||||
import org.junit.After;
|
||||
|
||||
public class TestCleanerWithEmbeddedTimelineServer extends TestCleaner {
|
||||
|
||||
@Override
|
||||
HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
HoodieWriteConfig.Builder builder = super.getConfigBuilder();
|
||||
try {
|
||||
builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
}
|
||||
@@ -61,7 +61,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
|
||||
.build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
/**
|
||||
|
||||
@@ -75,6 +75,28 @@ public class TestHoodieClientBase implements Serializable {
|
||||
protected TemporaryFolder folder = null;
|
||||
protected transient HoodieTestDataGenerator dataGen = null;
|
||||
|
||||
private HoodieWriteClient writeClient;
|
||||
|
||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws Exception {
|
||||
closeClient();
|
||||
writeClient = new HoodieWriteClient(jsc, cfg);
|
||||
return writeClient;
|
||||
}
|
||||
|
||||
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit)
|
||||
throws Exception {
|
||||
closeClient();
|
||||
writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit);
|
||||
return writeClient;
|
||||
}
|
||||
|
||||
private void closeClient() {
|
||||
if (null != writeClient) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
// Initialize a local spark env
|
||||
@@ -105,6 +127,8 @@ public class TestHoodieClientBase implements Serializable {
|
||||
* Properly release resources at end of each test
|
||||
*/
|
||||
public void tearDown() throws IOException {
|
||||
closeClient();
|
||||
|
||||
if (null != sqlContext) {
|
||||
logger.info("Clearing sql context cache of spark-session used in previous test-case");
|
||||
sqlContext.clearCache();
|
||||
|
||||
@@ -133,7 +133,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
boolean isPrepped) throws Exception {
|
||||
// Set autoCommit false
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
String prevCommitTime = "000";
|
||||
String newCommitTime = "001";
|
||||
@@ -428,7 +428,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
|
||||
|
||||
HoodieTable table = getHoodieTable(metadata, config);
|
||||
TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
|
||||
List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
|
||||
.collect(Collectors.toList());
|
||||
@@ -467,6 +468,10 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
assertEquals("Total inserts in commit3 must add up", keys3.size(), numTotalInsertsInCommit3);
|
||||
}
|
||||
|
||||
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
return HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test scenario of new file-group getting added during insert()
|
||||
*/
|
||||
@@ -532,7 +537,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieTable table = getHoodieTable(metaClient, config);
|
||||
List<HoodieDataFile> files = table.getROFileSystemView()
|
||||
.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
|
||||
.collect(Collectors.toList());
|
||||
@@ -555,7 +560,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
public void testCommitWritesRelativePaths() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
@@ -602,7 +607,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
public void testRollingStatsInMetadata() throws Exception {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
|
||||
@@ -674,7 +679,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
@Test
|
||||
public void testConsistencyCheckDuringFinalize() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieWriteClient client = getHoodieWriteClient(cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
|
||||
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.IOException;
|
||||
import org.junit.After;
|
||||
|
||||
public class TestHoodieReadClientWithEmbeddedServer extends TestHoodieReadClient {
|
||||
|
||||
@Override
|
||||
HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
HoodieWriteConfig.Builder builder = super.getConfigBuilder();
|
||||
try {
|
||||
builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import org.junit.After;
|
||||
|
||||
public class TestHoodieWriteClientWithEmbeddedServer extends TestHoodieClientOnCopyOnWriteStorage {
|
||||
|
||||
@Override
|
||||
HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
HoodieWriteConfig.Builder builder = super.getConfigBuilder();
|
||||
try {
|
||||
builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.build());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
((SyncableFileSystemView) (table.getRTFileSystemView())).reset();
|
||||
return table;
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
super.tearDown();
|
||||
}
|
||||
}
|
||||
@@ -60,7 +60,7 @@ public class TestMultiFS implements Serializable {
|
||||
private static JavaSparkContext jsc;
|
||||
private static SQLContext sqlContext;
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
private String tableName = "hoodie_rt";
|
||||
protected String tableName = "hoodie_rt";
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@BeforeClass
|
||||
@@ -96,6 +96,13 @@ public class TestMultiFS implements Serializable {
|
||||
FileSystem.closeAll();
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void readLocalWriteHDFS() throws Exception {
|
||||
|
||||
@@ -108,10 +115,7 @@ public class TestMultiFS implements Serializable {
|
||||
HoodieAvroPayload.class.getName());
|
||||
|
||||
//Create write client to write some records in
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(dfsBasePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
|
||||
HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg);
|
||||
|
||||
// Write generated data to hdfs (only inserts)
|
||||
@@ -132,10 +136,7 @@ public class TestMultiFS implements Serializable {
|
||||
HoodieTableMetaClient
|
||||
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
|
||||
HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig);
|
||||
|
||||
String writeCommitTime = localWriteClient.startCommit();
|
||||
@@ -151,5 +152,8 @@ public class TestMultiFS implements Serializable {
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> localReadRecords = HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
|
||||
assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
|
||||
|
||||
hdfsWriteClient.close();
|
||||
localWriteClient.close();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
|
||||
/**
|
||||
* Tests MultiFS with embedded timeline server enabled
|
||||
*/
|
||||
public class TestMultiFSWithEmbeddedServer extends TestMultiFS {
|
||||
|
||||
@Override
|
||||
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
}
|
||||
}
|
||||
@@ -24,7 +24,7 @@ public class TestHoodieDataFile extends HoodieDataFile {
|
||||
private final long size;
|
||||
|
||||
public TestHoodieDataFile(long size) {
|
||||
super(null);
|
||||
super("/tmp/XYXYXYXYXYYX_11_20180918020003.parquet");
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
|
||||
@@ -17,7 +17,6 @@
|
||||
package com.uber.hoodie.io.strategy;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
public class TestHoodieLogFile extends HoodieLogFile {
|
||||
@@ -25,7 +24,7 @@ public class TestHoodieLogFile extends HoodieLogFile {
|
||||
private final long size;
|
||||
|
||||
public TestHoodieLogFile(long size) {
|
||||
super((Path) null);
|
||||
super("/tmp/.ce481ee7-9e53-4a2e-9992-f9e295fa79c0_20180919184844.log.1");
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
@@ -39,7 +38,7 @@ public class TestHoodieLogFile extends HoodieLogFile {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Long> getFileSize() {
|
||||
return Optional.of(size);
|
||||
public long getFileSize() {
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.uber.hoodie.HoodieReadClient;
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
@@ -42,6 +43,7 @@ import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
@@ -53,7 +55,6 @@ import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.index.HoodieIndex.IndexType;
|
||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
@@ -83,7 +84,7 @@ import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestMergeOnReadTable {
|
||||
|
||||
private static String basePath = null;
|
||||
protected String basePath = null;
|
||||
//NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class)
|
||||
//The implementation and gurantees of many API's differ, for example check rename(src,dst)
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
@@ -97,6 +98,9 @@ public class TestMergeOnReadTable {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
dfsCluster = null;
|
||||
dfs = null;
|
||||
hdfsTestService = null;
|
||||
}
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
@@ -441,14 +445,7 @@ public class TestMergeOnReadTable {
|
||||
*/
|
||||
final String commitTime1 = "002";
|
||||
// WriteClient with custom config (disable small file handling)
|
||||
client = new HoodieWriteClient(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
|
||||
.forTable("test-trip-table").build());
|
||||
client = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
|
||||
client.startCommitWithTime(commitTime1);
|
||||
|
||||
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
||||
@@ -614,14 +611,7 @@ public class TestMergeOnReadTable {
|
||||
newCommitTime = "002";
|
||||
allCommits.add(newCommitTime);
|
||||
// WriteClient with custom config (disable small file handling)
|
||||
HoodieWriteClient nClient = new HoodieWriteClient(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
|
||||
.forTable("test-trip-table").build());
|
||||
HoodieWriteClient nClient = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
|
||||
nClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
|
||||
@@ -736,6 +726,16 @@ public class TestMergeOnReadTable {
|
||||
assertTrue(fileGroups.isEmpty());
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
|
||||
.forTable("test-trip-table").build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertPartitioner() throws Exception {
|
||||
@@ -834,13 +834,14 @@ public class TestMergeOnReadTable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
HoodieTimeline timeline2 = metaClient.getActiveTimeline();
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new HoodieBloomIndex<>(config);
|
||||
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, config);
|
||||
updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect();
|
||||
|
||||
// Write them to corresponding avro logfiles
|
||||
HoodieTestUtils
|
||||
@@ -850,6 +851,9 @@ public class TestMergeOnReadTable {
|
||||
// Verify that all data file has one log file
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
// In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
|
||||
((SyncableFileSystemView)(table.getRTFileSystemView())).reset();
|
||||
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
@@ -1065,6 +1069,9 @@ public class TestMergeOnReadTable {
|
||||
writeClient.commitCompaction(newCommitTime, statuses, Optional.empty());
|
||||
// Trigger a rollback of compaction
|
||||
writeClient.rollback(newCommitTime);
|
||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
tableRTFileSystemView = table.getRTFileSystemView();
|
||||
((SyncableFileSystemView)tableRTFileSystemView).reset();
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
|
||||
fileSlice.getDataFile().isPresent()).count() == 0);
|
||||
@@ -1292,13 +1299,14 @@ public class TestMergeOnReadTable {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
return getConfigBuilder(autoCommit, IndexType.BLOOM);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
|
||||
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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.uber.hoodie.table;
|
||||
|
||||
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
|
||||
public class TestMergeonReadTableWithEmbeddedServer extends TestMergeOnReadTable {
|
||||
|
||||
@Override
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
|
||||
.withEmbeddedTimelineServerEnabled(true)
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withEmbeddedTimelineServerEnabled(true)
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
|
||||
.forTable("test-trip-table").build();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user