1
0

[HUDI-242] Support for RFC-12/Bootstrapping of external datasets to hudi (#1876)

- [HUDI-418] Bootstrap Index Implementation using HFile with unit-test
 - [HUDI-421] FileSystem View Changes to support Bootstrap with unit-tests
 - [HUDI-424] Implement Query Side Integration for querying tables containing bootstrap file slices
 - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices
 - [HUDI-421] Bootstrap Write Client with tests
 - [HUDI-425] Added HoodieDeltaStreamer support
 - [HUDI-899] Add a knob to change partition-path style while performing metadata bootstrap
 - [HUDI-900] Metadata Bootstrap Key Generator needs to handle complex keys correctly
 - [HUDI-424] Simplify Record reader implementation
 - [HUDI-423] Implement upsert functionality for handling updates to these bootstrap file slices
 - [HUDI-420] Hoodie Demo working with hive and sparkSQL. Also, Hoodie CLI working with bootstrap tables

Co-authored-by: Mehrotra <uditme@amazon.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
Co-authored-by: Balaji Varadarajan <varadarb@uber.com>
This commit is contained in:
vinoth chandar
2020-08-03 20:19:21 -07:00
committed by GitHub
parent 266bce12b3
commit 539621bd33
175 changed files with 7540 additions and 779 deletions

View File

@@ -0,0 +1,180 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.bootstrap;
import org.apache.hudi.avro.model.HoodieFSPermission;
import org.apache.hudi.avro.model.HoodieFileStatus;
import org.apache.hudi.avro.model.HoodiePath;
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
import org.apache.hudi.common.bootstrap.index.BootstrapIndex.IndexWriter;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hadoop.fs.permission.FsAction;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
/**
* Unit Tests for Bootstrap Index.
*/
public class TestBootstrapIndex extends HoodieCommonTestHarness {
private static String[] PARTITIONS = {"2020/03/18", "2020/03/19", "2020/03/20", "2020/03/21"};
private static String BOOTSTRAP_BASE_PATH = "/tmp/source/parquet_tables/table1";
@BeforeEach
public void init() throws IOException {
initMetaClient();
}
@Test
public void testBootstrapIndex() throws IOException {
testBootstrapIndexOneRound(10);
}
@Test
public void testBootstrapIndexRecreateIndex() throws IOException {
testBootstrapIndexOneRound(10);
HFileBootstrapIndex index = new HFileBootstrapIndex(metaClient);
index.dropIndex();
// Run again this time recreating bootstrap index
testBootstrapIndexOneRound(5);
}
@Test
public void testBootstrapIndexConcurrent() throws Exception {
Map<String, List<BootstrapFileMapping>> bootstrapMapping = generateBootstrapIndex(100);
final int numThreads = 20;
final int numRequestsPerThread = 50;
ExecutorService service = Executors.newFixedThreadPool(numThreads);
try {
List<Future<Boolean>> futureList = new ArrayList<>();
for (int i = 0; i < numThreads; i++) {
Future<Boolean> result = service.submit(() -> {
for (int j = 0; j < numRequestsPerThread; j++) {
validateBootstrapIndex(bootstrapMapping);
}
return true;
});
futureList.add(result);
}
for (Future<Boolean> res : futureList) {
res.get();
}
} finally {
service.shutdownNow();
}
}
private void testBootstrapIndexOneRound(int numEntriesPerPartition) throws IOException {
Map<String, List<BootstrapFileMapping>> bootstrapMapping = generateBootstrapIndex(numEntriesPerPartition);
validateBootstrapIndex(bootstrapMapping);
}
private Map<String, List<BootstrapFileMapping>> generateBootstrapIndex(int numEntriesPerPartition)
throws IOException {
Map<String, List<BootstrapFileMapping>> bootstrapMapping = generateBootstrapMapping(numEntriesPerPartition);
BootstrapIndex index = new HFileBootstrapIndex(metaClient);
try (IndexWriter writer = index.createWriter(BOOTSTRAP_BASE_PATH)) {
writer.begin();
bootstrapMapping.entrySet().stream().forEach(e -> writer.appendNextPartition(e.getKey(), e.getValue()));
writer.finish();
}
return bootstrapMapping;
}
private void validateBootstrapIndex(Map<String, List<BootstrapFileMapping>> bootstrapMapping) {
BootstrapIndex index = new HFileBootstrapIndex(metaClient);
try (BootstrapIndex.IndexReader reader = index.createReader()) {
List<String> partitions = reader.getIndexedPartitionPaths();
assertEquals(bootstrapMapping.size(), partitions.size());
long expNumFileGroupKeys = bootstrapMapping.values().stream().flatMap(x -> x.stream()).count();
long gotNumFileGroupKeys = reader.getIndexedFileIds().size();
assertEquals(expNumFileGroupKeys, gotNumFileGroupKeys);
bootstrapMapping.entrySet().stream().forEach(e -> {
List<BootstrapFileMapping> gotMapping = reader.getSourceFileMappingForPartition(e.getKey());
List<BootstrapFileMapping> expected = new ArrayList<>(e.getValue());
Collections.sort(gotMapping);
Collections.sort(expected);
assertEquals(expected, gotMapping, "Check for bootstrap index entries for partition " + e.getKey());
List<HoodieFileGroupId> fileIds = e.getValue().stream().map(BootstrapFileMapping::getFileGroupId)
.collect(Collectors.toList());
Map<HoodieFileGroupId, BootstrapFileMapping> lookupResult = reader.getSourceFileMappingForFileIds(fileIds);
assertEquals(fileIds.size(), lookupResult.size());
e.getValue().forEach(x -> {
BootstrapFileMapping res = lookupResult.get(x.getFileGroupId());
assertNotNull(res);
assertEquals(x.getFileId(), res.getFileId());
assertEquals(x.getPartitionPath(), res.getPartitionPath());
assertEquals(BOOTSTRAP_BASE_PATH, res.getBootstrapBasePath());
assertEquals(x.getBoostrapFileStatus(), res.getBoostrapFileStatus());
assertEquals(x.getBootstrapPartitionPath(), res.getBootstrapPartitionPath());
});
});
}
}
private Map<String, List<BootstrapFileMapping>> generateBootstrapMapping(int numEntriesPerPartition) {
return Arrays.stream(PARTITIONS).map(partition -> {
return Pair.of(partition, IntStream.range(0, numEntriesPerPartition).mapToObj(idx -> {
String hudiFileId = UUID.randomUUID().toString();
String sourceFileName = idx + ".parquet";
HoodieFileStatus sourceFileStatus = HoodieFileStatus.newBuilder()
.setPath(HoodiePath.newBuilder().setUri(BOOTSTRAP_BASE_PATH + "/" + partition + "/" + sourceFileName).build())
.setLength(256 * 1024 * 1024L)
.setAccessTime(new Date().getTime())
.setModificationTime(new Date().getTime() + 99999)
.setBlockReplication(2)
.setOwner("hudi")
.setGroup("hudi")
.setBlockSize(128 * 1024 * 1024L)
.setPermission(HoodieFSPermission.newBuilder().setUserAction(FsAction.ALL.name())
.setGroupAction(FsAction.READ.name()).setOtherAction(FsAction.NONE.name()).setStickyBit(true).build())
.build();
return new BootstrapFileMapping(BOOTSTRAP_BASE_PATH, partition, partition, sourceFileStatus, hudiFileId);
}).collect(Collectors.toList()));
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
}
}

View File

@@ -19,7 +19,15 @@
package org.apache.hudi.common.table.view;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieFSPermission;
import org.apache.hudi.avro.model.HoodieFileStatus;
import org.apache.hudi.avro.model.HoodiePath;
import org.apache.hudi.common.bootstrap.FileStatusUtils;
import org.apache.hudi.common.bootstrap.index.BootstrapIndex.IndexWriter;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BaseFile;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
@@ -35,22 +43,29 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -70,16 +85,23 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
private static final Logger LOG = LogManager.getLogger(TestHoodieTableFileSystemView.class);
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with bootstrap enable={0}";
private static String TEST_WRITE_TOKEN = "1-0-1";
private static final String TEST_WRITE_TOKEN = "1-0-1";
private static final String BOOTSTRAP_SOURCE_PATH = "/usr/warehouse/hive/data/tables/src1/";
protected SyncableFileSystemView fsView;
protected BaseFileOnlyView roView;
protected SliceView rtView;
public static Stream<Arguments> configParams() {
return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of);
}
@BeforeEach
public void init() throws IOException {
initMetaClient();
public void setup() throws IOException {
metaClient = HoodieTestUtils.init(tempDir.toAbsolutePath().toString(), getTableType(), BOOTSTRAP_SOURCE_PATH);
basePath = metaClient.getBasePath();
refreshFsView();
}
@@ -128,6 +150,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
Paths.get(basePath, partitionPath, fileName1).toFile().createNewFile();
Paths.get(basePath, partitionPath, fileName2).toFile().createNewFile();
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1);
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
@@ -184,29 +207,33 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
"Total number of file-groups in view matches expected");
}
@Test
public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction() throws Exception {
testViewForFileSlicesWithAsyncCompaction(true, false, 2, 1, true);
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
@MethodSource("configParams")
public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction(boolean testBootstrap) throws Exception {
testViewForFileSlicesWithAsyncCompaction(true, false, 2, 1, true, testBootstrap);
}
@Test
public void testViewForFileSlicesWithBaseFileAndRequestedCompaction() throws Exception {
testViewForFileSlicesWithAsyncCompaction(false, false, 2, 2, true);
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
@MethodSource("configParams")
public void testViewForFileSlicesWithBaseFileAndRequestedCompaction(boolean testBootstrap) throws Exception {
testViewForFileSlicesWithAsyncCompaction(false, false, 2, 2, true, testBootstrap);
}
@Test
public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction() throws Exception {
testViewForFileSlicesWithAsyncCompaction(true, true, 2, 1, true);
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
@MethodSource("configParams")
public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction(boolean testBootstrap) throws Exception {
testViewForFileSlicesWithAsyncCompaction(true, true, 2, 1, true, testBootstrap);
}
@Test
public void testViewForFileSlicesWithBaseFileAndInflightCompaction() throws Exception {
testViewForFileSlicesWithAsyncCompaction(false, true, 2, 2, true);
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
@MethodSource("configParams")
public void testViewForFileSlicesWithBaseFileAndInflightCompaction(boolean testBootstrap) throws Exception {
testViewForFileSlicesWithAsyncCompaction(false, true, 2, 2, true, testBootstrap);
}
/**
* Returns all file-slices including uncommitted ones.
*
*
* @param partitionPath
* @return
*/
@@ -217,7 +244,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
/**
* Returns latest raw file-slices including uncommitted ones.
*
*
* @param partitionPath
* @return
*/
@@ -226,9 +253,36 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
.filter(Option::isPresent).map(Option::get);
}
private void checkExternalFile(HoodieFileStatus srcFileStatus, Option<BaseFile> bootstrapBaseFile, boolean testBootstrap) {
if (testBootstrap) {
assertTrue(bootstrapBaseFile.isPresent());
assertEquals(FileStatusUtils.toPath(srcFileStatus.getPath()), new Path(bootstrapBaseFile.get().getPath()));
assertEquals(srcFileStatus.getPath(), FileStatusUtils.fromPath(new Path(bootstrapBaseFile.get().getPath())));
assertEquals(srcFileStatus.getOwner(), bootstrapBaseFile.get().getFileStatus().getOwner());
assertEquals(srcFileStatus.getGroup(), bootstrapBaseFile.get().getFileStatus().getGroup());
assertEquals(srcFileStatus.getAccessTime(), new Long(bootstrapBaseFile.get().getFileStatus().getAccessTime()));
assertEquals(srcFileStatus.getModificationTime(),
new Long(bootstrapBaseFile.get().getFileStatus().getModificationTime()));
assertEquals(srcFileStatus.getBlockSize(), new Long(bootstrapBaseFile.get().getFileStatus().getBlockSize()));
assertEquals(srcFileStatus.getLength(), new Long(bootstrapBaseFile.get().getFileStatus().getLen()));
assertEquals(srcFileStatus.getBlockReplication(),
new Integer(bootstrapBaseFile.get().getFileStatus().getReplication()));
assertEquals(srcFileStatus.getIsDir() == null ? false : srcFileStatus.getIsDir(),
bootstrapBaseFile.get().getFileStatus().isDirectory());
assertEquals(FileStatusUtils.toFSPermission(srcFileStatus.getPermission()),
bootstrapBaseFile.get().getFileStatus().getPermission());
assertEquals(srcFileStatus.getPermission(),
FileStatusUtils.fromFSPermission(bootstrapBaseFile.get().getFileStatus().getPermission()));
assertEquals(srcFileStatus.getSymlink() != null,
bootstrapBaseFile.get().getFileStatus().isSymlink());
} else {
assertFalse(bootstrapBaseFile.isPresent());
}
}
/**
* Helper method to test Views in the presence of concurrent compaction.
*
*
* @param skipCreatingDataFile if set, first File Slice will not have data-file set. This would simulate inserts going
* directly to log files
* @param isCompactionInFlight if set, compaction was inflight (running) when view was tested first time, otherwise
@@ -236,16 +290,31 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
* @param expTotalFileSlices Total number of file-slices across file-groups in the partition path
* @param expTotalDataFiles Total number of data-files across file-groups in the partition path
* @param includeInvalidAndInflight Whether view includes inflight and invalid file-groups.
* @param testBootstrap enable Bootstrap and test
* @throws Exception -
*/
protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile, boolean isCompactionInFlight,
int expTotalFileSlices, int expTotalDataFiles, boolean includeInvalidAndInflight) throws Exception {
int expTotalFileSlices, int expTotalDataFiles, boolean includeInvalidAndInflight, boolean testBootstrap)
throws Exception {
String partitionPath = "2016/05/01";
new File(basePath + "/" + partitionPath).mkdirs();
String fileId = UUID.randomUUID().toString();
String srcName = "part_0000.parquet";
HoodieFileStatus srcFileStatus = HoodieFileStatus.newBuilder()
.setPath(HoodiePath.newBuilder().setUri(BOOTSTRAP_SOURCE_PATH + partitionPath + "/" + srcName).build())
.setLength(256 * 1024 * 1024L)
.setAccessTime(new Date().getTime())
.setModificationTime(new Date().getTime() + 99999)
.setBlockReplication(2)
.setOwner("hudi")
.setGroup("hudi")
.setBlockSize(128 * 1024 * 1024L)
.setPermission(HoodieFSPermission.newBuilder().setUserAction(FsAction.ALL.name())
.setGroupAction(FsAction.READ.name()).setOtherAction(FsAction.NONE.name()).setStickyBit(true).build())
.build();
// if skipCreatingDataFile, then instantTime1 below acts like delta-commit, otherwise it is base-commit
String instantTime1 = "1";
String instantTime1 = testBootstrap && !skipCreatingDataFile ? HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS : "1";
String deltaInstantTime1 = "2";
String deltaInstantTime2 = "3";
@@ -265,12 +334,30 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
if (testBootstrap && !skipCreatingDataFile) {
try (IndexWriter writer = new HFileBootstrapIndex(metaClient).createWriter(BOOTSTRAP_SOURCE_PATH)) {
writer.begin();
BootstrapFileMapping mapping = new BootstrapFileMapping(BOOTSTRAP_SOURCE_PATH, partitionPath,
partitionPath, srcFileStatus, fileId);
List<BootstrapFileMapping> b = new ArrayList<>();
b.add(mapping);
writer.appendNextPartition(partitionPath, b);
writer.finish();
}
}
saveAsComplete(commitTimeline, instant1, Option.empty());
saveAsComplete(commitTimeline, deltaInstant2, Option.empty());
saveAsComplete(commitTimeline, deltaInstant3, Option.empty());
refreshFsView();
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
assertEquals(1, fileSlices.size());
FileSlice fileSlice = fileSlices.get(0);
assertEquals(instantTime1, fileSlice.getBaseInstantTime());
if (!skipCreatingDataFile) {
assertTrue(fileSlice.getBaseFile().isPresent());
checkExternalFile(srcFileStatus, fileSlice.getBaseFile().get().getBootstrapBaseFile(), testBootstrap);
}
String compactionRequestedTime = "4";
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
List<Pair<String, FileSlice>> partitionFileSlicesPairs = new ArrayList<>();
@@ -328,10 +415,11 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
List<FileSlice> fileSliceList =
rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
assertEquals(1, fileSliceList.size(), "Expect file-slice to be merged");
FileSlice fileSlice = fileSliceList.get(0);
fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId());
if (!skipCreatingDataFile) {
assertEquals(dataFileName, fileSlice.getBaseFile().get().getFileName(), "Data file must be present");
checkExternalFile(srcFileStatus, fileSlice.getBaseFile().get().getBootstrapBaseFile(), testBootstrap);
} else {
assertFalse(fileSlice.getBaseFile().isPresent(), "No data-file expected as it was not created");
}
@@ -364,27 +452,34 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
} else {
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned"));
checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap);
}
dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList());
if (skipCreatingDataFile) {
assertEquals(0, dataFiles.size(), "Expect no data file to be returned");
} else {
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned"));
checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap);
}
dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
if (skipCreatingDataFile) {
assertEquals(0, dataFiles.size(), "Expect no data file to be returned");
} else {
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned"));
checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap);
}
dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList());
if (skipCreatingDataFile) {
assertEquals(0, dataFiles.size(), "Expect no data file to be returned");
} else {
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), instantTime1, "Expect data-file for instant 1 be returned"));
checkExternalFile(srcFileStatus, dataFiles.get(0).getBootstrapBaseFile(), testBootstrap);
}
// Inflight/Orphan File-groups needs to be in the view
@@ -499,6 +594,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals(1, dataFiles.size(), "Expect only one data-files in latest view as there is only one file-group");
assertEquals(compactDataFileName, dataFiles.get(0).getFileName(), "Data Filename must match");
assertEquals(1, fileSliceList.size(), "Only one latest file-slice in the partition");
assertFalse(dataFiles.get(0).getBootstrapBaseFile().isPresent(), "No external data file must be present");
fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId(), "Check file-Id is set correctly");
assertEquals(compactDataFileName, fileSlice.getBaseFile().get().getFileName(),
@@ -513,16 +609,30 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// Data Files API tests
dataFiles = roView.getLatestBaseFiles().collect(Collectors.toList());
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"));
assertFalse(dataFiles.get(0).getBootstrapBaseFile().isPresent(),"No external data file must be present");
dataFiles.forEach(df -> {
assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned");
assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present");
});
dataFiles = roView.getLatestBaseFiles(partitionPath).collect(Collectors.toList());
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"));
dataFiles.forEach(df -> {
assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned");
assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present");
});
dataFiles = roView.getLatestBaseFilesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"));
dataFiles.forEach(df -> {
assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned");
assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present");
});
dataFiles = roView.getLatestBaseFilesInRange(allInstantTimes).collect(Collectors.toList());
assertEquals(1, dataFiles.size(), "Expect only one data-file to be sent");
dataFiles.forEach(df -> assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned"));
dataFiles.forEach(df -> {
assertEquals(df.getCommitTime(), compactionRequestedTime, "Expect data-file created by compaction be returned");
assertFalse(df.getBootstrapBaseFile().isPresent(), "No external data file must be present");
});
assertEquals(expTotalFileSlices, rtView.getAllFileSlices(partitionPath).count(),
"Total number of file-slices in partitions matches expected");

View File

@@ -43,7 +43,13 @@ public class HoodieCommonTestHarness {
* Initializes basePath.
*/
protected void initPath() {
this.basePath = tempDir.toAbsolutePath().toString();
try {
java.nio.file.Path basePath = tempDir.resolve("dataset");
java.nio.file.Files.createDirectories(basePath);
this.basePath = basePath.toString();
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}
/**

View File

@@ -19,6 +19,9 @@
package org.apache.hudi.common.testutils;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.generic.GenericFixed;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.fs.FSUtils;
@@ -34,12 +37,9 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -64,6 +64,7 @@ import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
@@ -107,7 +108,7 @@ public class HoodieTestDataGenerator {
+ "{\"name\": \"nation\", \"type\": \"bytes\"},"
+ "{\"name\": \"user_defined_delete_marker_field\", \"type\": \"boolean\", \"default\": false},"
+ "{\"name\":\"current_date\",\"type\": {\"type\": \"int\", \"logicalType\": \"date\"}},"
+ "{\"name\":\"current_ts\",\"type\": {\"type\": \"long\", \"logicalType\": \"timestamp-micros\"}},"
+ "{\"name\":\"current_ts\",\"type\": {\"type\": \"long\"}},"
+ "{\"name\":\"height\",\"type\":{\"type\":\"fixed\",\"name\":\"abc\",\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":6}},";
public static final String TRIP_EXAMPLE_SCHEMA =
@@ -126,6 +127,7 @@ public class HoodieTestDataGenerator {
public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,int,bigint,float,binary,boolean,int,bigint,decimal(10,6),"
+ "map<string,string>,struct<amount:double,currency:string>,array<struct<amount:double,currency:string>>,boolean";
public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS =
HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA);
@@ -404,7 +406,7 @@ public class HoodieTestDataGenerator {
}
public List<HoodieRecord> generateInsertsAsPerSchema(String commitTime, Integer n, String schemaStr) {
return generateInsertsStream(commitTime, n, schemaStr).collect(Collectors.toList());
return generateInsertsStream(commitTime, n, false, schemaStr).collect(Collectors.toList());
}
/**
@@ -431,38 +433,35 @@ public class HoodieTestDataGenerator {
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n, String schemaStr) {
return generateInsertsStream(commitTime, n, false, schemaStr);
public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n, boolean isFlattened, String schemaStr) {
return generateInsertsStream(commitTime, n, isFlattened, schemaStr, false);
}
public List<HoodieRecord> generateInsertsContainsAllPartitions(String instantTime, Integer n) {
if (n < partitionPaths.length) {
throw new HoodieIOException("n must greater then partitionPaths length");
}
return generateInsertsStream(
instantTime, n, false, TRIP_EXAMPLE_SCHEMA, true).collect(Collectors.toList());
return generateInsertsStream(instantTime, n, false, TRIP_EXAMPLE_SCHEMA, true).collect(Collectors.toList());
}
public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions) {
return generateInsertsStream(commitTime, n, isFlattened, schemaStr, containsAllPartitions,
() -> partitionPaths[RAND.nextInt(partitionPaths.length)],
() -> UUID.randomUUID().toString());
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public Stream<HoodieRecord> generateInsertsStream(
String instantTime, Integer n, boolean isFlattened, String schemaStr) {
return generateInsertsStream(instantTime, n, isFlattened, schemaStr, false);
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public Stream<HoodieRecord> generateInsertsStream(
String instantTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions) {
public Stream<HoodieRecord> generateInsertsStream(String instantTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions,
Supplier<String> partitionPathSupplier, Supplier<String> recordKeySupplier) {
int currSize = getNumExistingKeys(schemaStr);
return IntStream.range(0, n).boxed().map(i -> {
String partitionPath = partitionPaths[RAND.nextInt(partitionPaths.length)];
String partitionPath = partitionPathSupplier.get();
if (containsAllPartitions && i < partitionPaths.length) {
partitionPath = partitionPaths[i];
}
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
HoodieKey key = new HoodieKey(recordKeySupplier.get(), partitionPath);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;

View File

@@ -118,6 +118,12 @@ public class HoodieTestUtils {
return init(getDefaultHadoopConf(), basePath, tableType);
}
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath) throws IOException {
Properties props = new Properties();
props.setProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath);
return init(getDefaultHadoopConf(), basePath, tableType, props);
}
public static HoodieTableMetaClient init(String basePath, HoodieFileFormat baseFileFormat) throws IOException {
return init(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE, baseFileFormat);
}

View File

@@ -17,6 +17,7 @@
###
log4j.rootLogger=WARN, CONSOLE
log4j.logger.org.apache.hudi=DEBUG
log4j.logger.org.apache.hadoop.hbase=ERROR
# CONSOLE is set to be a ConsoleAppender.
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender

View File

@@ -18,6 +18,7 @@
log4j.rootLogger=WARN, CONSOLE
log4j.logger.org.apache=INFO
log4j.logger.org.apache.hudi=DEBUG
log4j.logger.org.apache.hadoop.hbase=ERROR
# A1 is set to be a ConsoleAppender.
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender