1
0

Perform consistency checks during write finalize

- Check to ensure written files are listable on storage
 - Docs reflected to capture how this helps with s3 storage
 - Unit tests added, corrections to existing tests
 - Fix DeltaStreamer to manage archived commits in a separate folder
This commit is contained in:
vinothchandar
2018-09-20 17:50:27 +05:30
committed by vinoth chandar
parent 4c74dd4cad
commit 9ca6f91e97
17 changed files with 381 additions and 93 deletions

View File

@@ -192,6 +192,7 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
.retainFileVersions(maxVersions).build())
.withParallelism(1, 1).withBulkInsertParallelism(1)
.withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true)
.build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
@@ -271,9 +272,6 @@ public class TestCleaner extends TestHoodieClientBase {
for (HoodieFileGroup fileGroup : fileGroups) {
if (selectedFileIdForCompaction.containsKey(fileGroup.getId())) {
// Ensure latest file-slice selected for compaction is retained
String oldestCommitRetained =
fileGroup.getAllDataFiles().map(HoodieDataFile::getCommitTime).sorted().findFirst().get();
Optional<HoodieDataFile> dataFileForCompactionPresent =
fileGroup.getAllDataFiles().filter(df -> {
return compactionFileIdToLatestFileSlice.get(fileGroup.getId())
@@ -357,7 +355,8 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainCommits(maxCommits).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).build();
.withParallelism(1, 1).withBulkInsertParallelism(1)
.withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =

View File

@@ -143,6 +143,8 @@ public class TestHoodieClientBase implements Serializable {
HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.withBulkInsertParallelism(2).withFinalizeWriteParallelism(2)
.withConsistencyCheckEnabled(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table")

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -40,6 +41,8 @@ import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.FileInputStream;
@@ -665,6 +668,42 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
}
/**
* Tests behavior of committing only when consistency is verified
*/
@Test
public void testConsistencyCheckDuringFinalize() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
String commitTime = "000";
client.startCommitWithTime(commitTime);
JavaRDD<HoodieRecord> writeRecords = jsc
.parallelize(dataGen.generateInserts(commitTime, 200), 1);
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
// move one of the files & commit should fail
WriteStatus status = result.take(1).get(0);
Path origPath = new Path(basePath + "/" + status.getStat().getPath());
Path hidePath = new Path(basePath + "/" + status.getStat().getPath() + "_hide");
metaClient.getFs().rename(origPath, hidePath);
try {
client.commit(commitTime, result);
fail("Commit should fail due to consistency check");
} catch (HoodieCommitException cme) {
assertTrue(cme.getCause() instanceof HoodieIOException);
}
// Re-introduce & commit should succeed
metaClient.getFs().rename(hidePath, origPath);
assertTrue("Commit should succeed", client.commit(commitTime, result));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
}
/**
* Build Hoodie Write Config for small data file sizes
*/

View File

@@ -24,6 +24,7 @@ import static org.mockito.Mockito.times;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTestUtils;
@@ -50,7 +51,6 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
@@ -101,8 +101,7 @@ public class TestHbaseIndex {
hbaseConfig = utility.getConnection().getConfiguration();
utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s"));
// Initialize a local spark env
SparkConf sparkConf = new SparkConf().setAppName("TestHbaseIndex").setMaster("local[1]");
jsc = new JavaSparkContext(sparkConf);
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHbaseIndex"));
jsc.hadoopConfiguration().addResource(utility.getConfiguration());
}

View File

@@ -0,0 +1,91 @@
/*
* 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.io;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.anyInt;
import static org.mockito.Mockito.anyList;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import com.uber.hoodie.common.HoodieClientTestUtils;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestConsistencyCheck {
private String basePath;
private JavaSparkContext jsc;
@Before
public void setup() throws IOException {
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("ConsistencyCheckTest"));
TemporaryFolder testFolder = new TemporaryFolder();
testFolder.create();
basePath = testFolder.getRoot().getAbsolutePath();
}
@After
public void teardown() {
if (jsc != null) {
jsc.stop();
}
File testFolderPath = new File(basePath);
if (testFolderPath.exists()) {
testFolderPath.delete();
}
}
@Test
public void testExponentialBackoff() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
JavaSparkContext jscSpy = spy(jsc);
ConsistencyCheck failing = new ConsistencyCheck(basePath,
Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f2_0_000.parquet"),
jscSpy, 2);
long startMs = System.currentTimeMillis();
assertEquals(1, failing.check(5, 10).size());
assertTrue((System.currentTimeMillis() - startMs) > (10 + 20 + 40 + 80));
verify(jscSpy, times(5)).parallelize(anyList(), anyInt());
}
@Test
public void testCheckPassingAndFailing() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f2");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f3");
ConsistencyCheck passing = new ConsistencyCheck(basePath,
Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f2_0_000.parquet"),
jsc, 2);
assertEquals(0, passing.check(1, 1000).size());
ConsistencyCheck failing = new ConsistencyCheck(basePath,
Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f4_0_000.parquet"),
jsc, 2);
assertEquals(1, failing.check(1, 1000).size());
}
}