HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.hadoop;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.lang.annotation.Annotation;
|
||||
import org.junit.Test;
|
||||
|
||||
public class AnnotationTest {
|
||||
|
||||
@Test
|
||||
public void testAnnotation() {
|
||||
assertTrue(HoodieInputFormat.class.isAnnotationPresent(UseFileSplitsFromInputFormat.class));
|
||||
Annotation[] annotations = HoodieInputFormat.class.getAnnotations();
|
||||
boolean found = false;
|
||||
for (Annotation annotation : annotations) {
|
||||
if ("UseFileSplitsFromInputFormat".equals(annotation.annotationType().getSimpleName())) {
|
||||
found = true;
|
||||
}
|
||||
}
|
||||
assertTrue(found);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,248 @@
|
||||
/*
|
||||
* 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.hadoop;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class HoodieInputFormatTest {
|
||||
|
||||
private HoodieInputFormat inputFormat;
|
||||
private JobConf jobConf;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
inputFormat = new HoodieInputFormat();
|
||||
jobConf = new JobConf();
|
||||
inputFormat.setConf(jobConf);
|
||||
}
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder basePath = new TemporaryFolder();
|
||||
|
||||
@Test
|
||||
public void testInputFormatLoad() throws IOException {
|
||||
// initial commit
|
||||
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
|
||||
InputFormatTestUtil.commit(basePath, "100");
|
||||
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
|
||||
InputSplit[] inputSplits = inputFormat.getSplits(jobConf, 10);
|
||||
assertEquals(10, inputSplits.length);
|
||||
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
assertEquals(10, files.length);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputFormatUpdates() throws IOException {
|
||||
// initial commit
|
||||
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
|
||||
InputFormatTestUtil.commit(basePath, "100");
|
||||
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
assertEquals(10, files.length);
|
||||
|
||||
// update files
|
||||
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 5, "200", true);
|
||||
// Before the commit
|
||||
files = inputFormat.listStatus(jobConf);
|
||||
assertEquals(10, files.length);
|
||||
ensureFilesInCommit(
|
||||
"Commit 200 has not been committed. We should not see files from this commit", files, "200",
|
||||
0);
|
||||
InputFormatTestUtil.commit(basePath, "200");
|
||||
files = inputFormat.listStatus(jobConf);
|
||||
assertEquals(10, files.length);
|
||||
ensureFilesInCommit(
|
||||
"5 files have been updated to commit 200. We should see 5 files from commit 200 and 5 "
|
||||
+ "files from 100 commit", files, "200", 5);
|
||||
ensureFilesInCommit(
|
||||
"5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 "
|
||||
+ "files from 200 commit", files, "100", 5);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIncrementalSimple() throws IOException {
|
||||
// initial commit
|
||||
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
|
||||
InputFormatTestUtil.commit(basePath, "100");
|
||||
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
|
||||
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
assertEquals(
|
||||
"We should exclude commit 100 when returning incremental pull with start commit time as "
|
||||
+ "100", 0, files.length);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIncrementalWithMultipleCommits() throws IOException {
|
||||
// initial commit
|
||||
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
|
||||
InputFormatTestUtil.commit(basePath, "100");
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
// update files
|
||||
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 5, "200", false);
|
||||
InputFormatTestUtil.commit(basePath, "200");
|
||||
|
||||
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 4, "300", false);
|
||||
InputFormatTestUtil.commit(basePath, "300");
|
||||
|
||||
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 3, "400", false);
|
||||
InputFormatTestUtil.commit(basePath, "400");
|
||||
|
||||
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 2, "500", false);
|
||||
InputFormatTestUtil.commit(basePath, "500");
|
||||
|
||||
InputFormatTestUtil.simulateUpdates(partitionDir, "100", 1, "600", false);
|
||||
InputFormatTestUtil.commit(basePath, "600");
|
||||
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
assertEquals("Pulling 1 commit from 100, should get us the 5 files committed at 200", 5,
|
||||
files.length);
|
||||
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 5 files committed at 200",
|
||||
files, "200", 5);
|
||||
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "100", 3);
|
||||
files = inputFormat.listStatus(jobConf);
|
||||
|
||||
assertEquals(
|
||||
"Pulling 3 commits from 100, should get us the 3 files from 400 commit, 1 file from 300 "
|
||||
+ "commit and 1 file from 200 commit", 5, files.length);
|
||||
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit",
|
||||
files, "400", 3);
|
||||
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit",
|
||||
files, "300", 1);
|
||||
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 200 commit",
|
||||
files, "200", 1);
|
||||
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "100", HoodieHiveUtil.MAX_COMMIT_ALL);
|
||||
files = inputFormat.listStatus(jobConf);
|
||||
|
||||
assertEquals(
|
||||
"Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 "
|
||||
+ "commits", 5, files.length);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 600 commit",
|
||||
files, "600", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 500 commit",
|
||||
files, "500", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 400 commit",
|
||||
files, "400", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 300 commit",
|
||||
files, "300", 1);
|
||||
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 200 commit",
|
||||
files, "200", 1);
|
||||
}
|
||||
|
||||
//TODO enable this after enabling predicate pushdown
|
||||
public void testPredicatePushDown() throws IOException {
|
||||
// initial commit
|
||||
Schema schema = InputFormatTestUtil.readSchema("/sample1.avsc");
|
||||
String commit1 = "20160628071126";
|
||||
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
|
||||
InputFormatTestUtil.commit(basePath, commit1);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
// check whether we have 10 records at this point
|
||||
ensureRecordsInCommit("We need to have 10 records at this point for commit " + commit1, commit1,
|
||||
10, 10);
|
||||
|
||||
// update 2 records in the original parquet file and save it as commit 200
|
||||
String commit2 = "20160629193623";
|
||||
InputFormatTestUtil.simulateParquetUpdates(partitionDir, schema, commit1, 10, 2, commit2);
|
||||
InputFormatTestUtil.commit(basePath, commit2);
|
||||
|
||||
InputFormatTestUtil.setupIncremental(jobConf, commit1, 1);
|
||||
// check whether we have 2 records at this point
|
||||
ensureRecordsInCommit(
|
||||
"We need to have 2 records that was modified at commit " + commit2 + " and no more",
|
||||
commit2, 2, 2);
|
||||
// Make sure we have the 10 records if we roll back the stattime
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "0", 2);
|
||||
ensureRecordsInCommit(
|
||||
"We need to have 8 records that was modified at commit " + commit1 + " and no more",
|
||||
commit1, 8, 10);
|
||||
ensureRecordsInCommit(
|
||||
"We need to have 2 records that was modified at commit " + commit2 + " and no more",
|
||||
commit2, 2, 10);
|
||||
}
|
||||
|
||||
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit,
|
||||
int totalExpected) throws IOException {
|
||||
int actualCount = 0;
|
||||
int totalCount = 0;
|
||||
InputSplit[] splits = inputFormat.getSplits(jobConf, 1);
|
||||
for (InputSplit split : splits) {
|
||||
RecordReader<NullWritable, ArrayWritable> recordReader = inputFormat
|
||||
.getRecordReader(split, jobConf, null);
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable writable = recordReader.createValue();
|
||||
|
||||
while (recordReader.next(key, writable)) {
|
||||
// writable returns an array with [field1, field2, _hoodie_commit_time,
|
||||
// _hoodie_commit_seqno]
|
||||
// Take the commit time and compare with the one we are interested in
|
||||
if (commit.equals((writable.get()[2]).toString())) {
|
||||
actualCount++;
|
||||
}
|
||||
totalCount++;
|
||||
}
|
||||
}
|
||||
assertEquals(msg, expectedNumberOfRecordsInCommit, actualCount);
|
||||
assertEquals(msg, totalExpected, totalCount);
|
||||
}
|
||||
|
||||
public static void ensureFilesInCommit(String msg, FileStatus[] files, String commit,
|
||||
int expected) {
|
||||
int count = 0;
|
||||
for (FileStatus file : files) {
|
||||
String commitTs = FSUtils.getCommitTime(file.getPath().getName());
|
||||
if (commit.equals(commitTs)) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
assertEquals(msg, expected, count);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,221 @@
|
||||
/*
|
||||
* 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.hadoop;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FilenameFilter;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.apache.parquet.avro.AvroParquetWriter;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class InputFormatTestUtil {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles,
|
||||
String commitNumber) throws IOException {
|
||||
basePath.create();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
File dataFile = new File(partitionPath,
|
||||
FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i));
|
||||
dataFile.createNewFile();
|
||||
}
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public static void simulateUpdates(File directory, final String originalCommit,
|
||||
int numberOfFilesUpdated, String newCommit, boolean randomize) throws IOException {
|
||||
List<File> dataFiles = Arrays.asList(directory.listFiles(new FilenameFilter() {
|
||||
@Override
|
||||
public boolean accept(File dir, String name) {
|
||||
String commitTs = FSUtils.getCommitTime(name);
|
||||
return originalCommit.equals(commitTs);
|
||||
}
|
||||
}));
|
||||
if (randomize) {
|
||||
Collections.shuffle(dataFiles);
|
||||
}
|
||||
List<File> toUpdateList = dataFiles
|
||||
.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
|
||||
for (File file : toUpdateList) {
|
||||
String fileId = FSUtils.getFileId(file.getName());
|
||||
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId));
|
||||
dataFile.createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static void commit(TemporaryFolder basePath, String commitNumber) throws IOException {
|
||||
// create the commit
|
||||
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".commit").createNewFile();
|
||||
}
|
||||
|
||||
public static void deltaCommit(TemporaryFolder basePath, String commitNumber) throws IOException {
|
||||
// create the commit
|
||||
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".deltacommit").createNewFile();
|
||||
}
|
||||
|
||||
public static void setupIncremental(JobConf jobConf, String startCommit,
|
||||
int numberOfCommitsToPull) {
|
||||
String modePropertyName = String
|
||||
.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE);
|
||||
|
||||
String startCommitTimestampName = String
|
||||
.format(HoodieHiveUtil.HOODIE_START_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
jobConf.set(startCommitTimestampName, startCommit);
|
||||
|
||||
String maxCommitPulls = String
|
||||
.format(HoodieHiveUtil.HOODIE_MAX_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
jobConf.setInt(maxCommitPulls, numberOfCommitsToPull);
|
||||
}
|
||||
|
||||
public static Schema readSchema(String location) throws IOException {
|
||||
return new Schema.Parser().parse(InputFormatTestUtil.class.getResourceAsStream(location));
|
||||
}
|
||||
|
||||
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema,
|
||||
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
|
||||
basePath.create();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
createData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
|
||||
public static File prepareSimpleParquetDataset(TemporaryFolder basePath, Schema schema,
|
||||
int numberOfFiles, int numberOfRecords, String commitNumber) throws Exception {
|
||||
basePath.create();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
createSimpleData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema,
|
||||
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
|
||||
baseDir.create();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), baseDir.getRoot().toString());
|
||||
File basePath = baseDir.getRoot();
|
||||
createData(schema, basePath, numberOfFiles, numberOfRecords, commitNumber);
|
||||
return basePath;
|
||||
}
|
||||
|
||||
private static void createData(Schema schema,
|
||||
File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber)
|
||||
throws IOException {
|
||||
AvroParquetWriter parquetWriter;
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i);
|
||||
File dataFile = new File(partitionPath, fileId);
|
||||
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
|
||||
try {
|
||||
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber,
|
||||
fileId)) {
|
||||
parquetWriter.write(record);
|
||||
}
|
||||
} finally {
|
||||
parquetWriter.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void createSimpleData(Schema schema,
|
||||
File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber)
|
||||
throws Exception {
|
||||
AvroParquetWriter parquetWriter;
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i);
|
||||
File dataFile = new File(partitionPath, fileId);
|
||||
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
|
||||
try {
|
||||
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, numberOfRecords);
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(schema);
|
||||
for (IndexedRecord record : records) {
|
||||
GenericRecord p = HoodieAvroUtils.rewriteRecord((GenericRecord) record, hoodieFieldsSchema);
|
||||
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
|
||||
p.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
|
||||
p.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitNumber);
|
||||
parquetWriter.write(p);
|
||||
}
|
||||
} finally {
|
||||
parquetWriter.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static Iterable<? extends GenericRecord> generateAvroRecords(Schema schema,
|
||||
int numberOfRecords, String commitTime, String fileId) throws IOException {
|
||||
List<GenericRecord> records = new ArrayList<>(numberOfRecords);
|
||||
for (int i = 0; i < numberOfRecords; i++) {
|
||||
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, commitTime, fileId));
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
public static void simulateParquetUpdates(File directory, Schema schema, String originalCommit,
|
||||
int totalNumberOfRecords, int numberOfRecordsToUpdate, String newCommit) throws IOException {
|
||||
File fileToUpdate = directory.listFiles(new FilenameFilter() {
|
||||
@Override
|
||||
public boolean accept(File dir, String name) {
|
||||
return name.endsWith("parquet");
|
||||
}
|
||||
})[0];
|
||||
String fileId = FSUtils.getFileId(fileToUpdate.getName());
|
||||
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId));
|
||||
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
|
||||
schema);
|
||||
try {
|
||||
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit,
|
||||
fileId)) {
|
||||
if (numberOfRecordsToUpdate > 0) {
|
||||
// update this record
|
||||
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, newCommit);
|
||||
String oldSeqNo = (String) record.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD);
|
||||
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
|
||||
oldSeqNo.replace(originalCommit, newCommit));
|
||||
numberOfRecordsToUpdate--;
|
||||
}
|
||||
parquetWriter.write(record);
|
||||
}
|
||||
} finally {
|
||||
parquetWriter.close();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,109 @@
|
||||
/*
|
||||
* 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.hadoop;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
|
||||
/**
|
||||
*/
|
||||
public class TestHoodieROTablePathFilter {
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
this.metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHoodiePaths() throws IOException {
|
||||
// Create a temp folder as the base path
|
||||
String basePath = metaClient.getBasePath();
|
||||
|
||||
HoodieTestUtils.createCommitFiles(basePath, "001", "002");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "003");
|
||||
HoodieTestUtils.createCompactionRequest(metaClient, "004", new ArrayList<>());
|
||||
|
||||
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f1");
|
||||
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f2");
|
||||
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "001", "f3");
|
||||
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "002", "f2");
|
||||
HoodieTestUtils.createDataFile(basePath, "2017/01/01", "003", "f3");
|
||||
|
||||
HoodieROTablePathFilter pathFilter = new HoodieROTablePathFilter();
|
||||
Path partitionPath = new Path("file://" + basePath + File.separator + "2017/01/01");
|
||||
assertTrue("Directories should be accepted", pathFilter.accept(partitionPath));
|
||||
|
||||
assertTrue(pathFilter.accept(new Path(
|
||||
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f1"))));
|
||||
assertFalse(pathFilter.accept(new Path(
|
||||
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f2"))));
|
||||
assertTrue(pathFilter.accept(new Path(
|
||||
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f3"))));
|
||||
assertTrue(pathFilter.accept(new Path(
|
||||
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "002", "f2"))));
|
||||
assertFalse(pathFilter.accept(new Path(
|
||||
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
|
||||
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "001"))));
|
||||
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "002"))));
|
||||
assertFalse(pathFilter.accept(new Path("file:///"
|
||||
+ HoodieTestUtils.getInflightCommitFilePath(basePath, "003"))));
|
||||
assertFalse(pathFilter.accept(new Path("file:///"
|
||||
+ HoodieTestUtils.getRequestedCompactionFilePath(basePath, "004"))));
|
||||
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/"
|
||||
+ HoodieTableMetaClient.METAFOLDER_NAME + "/")));
|
||||
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME)));
|
||||
|
||||
assertFalse(pathFilter.accept(new Path(
|
||||
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNonHoodiePaths() throws IOException {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
String basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieROTablePathFilter pathFilter = new HoodieROTablePathFilter();
|
||||
|
||||
String path = basePath + File.separator + "nonhoodiefolder";
|
||||
new File(path).mkdirs();
|
||||
assertTrue(pathFilter.accept(new Path("file:///" + path)));
|
||||
|
||||
path = basePath + File.separator + "nonhoodiefolder/somefile";
|
||||
new File(path).createNewFile();
|
||||
assertTrue(pathFilter.accept(new Path("file:///" + path)));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,105 @@
|
||||
/*
|
||||
* 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.hadoop;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRecordReaderValueIterator {
|
||||
|
||||
@Test
|
||||
public void testValueIterator() {
|
||||
String[] values = new String[]{
|
||||
"hoodie",
|
||||
"efficient",
|
||||
"new project",
|
||||
"realtime",
|
||||
"spark",
|
||||
"dataset",
|
||||
};
|
||||
List<Pair<Integer, String>> entries = IntStream.range(0, values.length)
|
||||
.boxed().map(idx -> Pair.of(idx, values[idx])).collect(Collectors.toList());
|
||||
TestRecordReader reader = new TestRecordReader(entries);
|
||||
RecordReaderValueIterator<IntWritable, Text> itr = new RecordReaderValueIterator<IntWritable, Text>(reader);
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Assert.assertTrue(itr.hasNext());
|
||||
Text val = itr.next();
|
||||
Assert.assertEquals(values[i], val.toString());
|
||||
}
|
||||
Assert.assertFalse(itr.hasNext());
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple replay record reader for unit-testing
|
||||
*/
|
||||
private static class TestRecordReader implements RecordReader<IntWritable, Text> {
|
||||
|
||||
private final List<Pair<Integer, String>> entries;
|
||||
private int currIndex = 0;
|
||||
|
||||
public TestRecordReader(List<Pair<Integer, String>> entries) {
|
||||
this.entries = entries;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean next(IntWritable key, Text value) throws IOException {
|
||||
if (currIndex >= entries.size()) {
|
||||
return false;
|
||||
}
|
||||
key.set(entries.get(currIndex).getLeft());
|
||||
value.set(entries.get(currIndex).getRight());
|
||||
currIndex++;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntWritable createKey() {
|
||||
return new IntWritable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Text createValue() {
|
||||
return new Text();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
return currIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getProgress() throws IOException {
|
||||
return (currIndex * 1.0F) / entries.size();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,571 @@
|
||||
/*
|
||||
* 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.hadoop.realtime;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.BooleanWritable;
|
||||
import org.apache.hadoop.io.DoubleWritable;
|
||||
import org.apache.hadoop.io.FloatWritable;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hadoop.InputFormatTestUtil;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class HoodieRealtimeRecordReaderTest {
|
||||
|
||||
private JobConf jobConf;
|
||||
private FileSystem fs;
|
||||
private Configuration hadoopConf;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
jobConf = new JobConf();
|
||||
jobConf.set(AbstractRealtimeRecordReader.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1 * 1024 * 1024));
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
fs = FSUtils.getFs(basePath.getRoot().getAbsolutePath(), hadoopConf);
|
||||
}
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder basePath = new TemporaryFolder();
|
||||
|
||||
private Writer writeLogFile(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, int numberOfRecords)
|
||||
throws InterruptedException, IOException {
|
||||
return writeDataBlockToLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0, 0);
|
||||
}
|
||||
|
||||
private Writer writeRollback(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, String rolledBackInstant, int logVersion)
|
||||
throws InterruptedException, IOException {
|
||||
Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(partitionDir.getPath()))
|
||||
.withFileId(fileId).overBaseCommit(baseCommit)
|
||||
.withFs(fs)
|
||||
.withLogVersion(logVersion)
|
||||
.withLogWriteToken("1-0-1")
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
// generate metadata
|
||||
Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HeaderMetadataType.INSTANT_TIME, newCommit);
|
||||
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, rolledBackInstant);
|
||||
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
|
||||
.ordinal()));
|
||||
// if update belongs to an existing log file
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(header));
|
||||
return writer;
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, int numberOfRecords, int offset, int logVersion)
|
||||
throws InterruptedException, IOException {
|
||||
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(partitionDir.getPath()))
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
|
||||
.withLogVersion(logVersion)
|
||||
.withLogWriteToken("1-0-1")
|
||||
.overBaseCommit(baseCommit).withFs(fs).build();
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
for (int i = offset; i < offset + numberOfRecords; i++) {
|
||||
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
|
||||
}
|
||||
Schema writeSchema = records.get(0).getSchema();
|
||||
Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString());
|
||||
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
|
||||
writer = writer.appendBlock(dataBlock);
|
||||
return writer;
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionDir, Schema schema, String fileId,
|
||||
String baseCommit, String newCommit, String oldCommit, int logVersion)
|
||||
throws InterruptedException, IOException {
|
||||
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(partitionDir.getPath()))
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
|
||||
.overBaseCommit(baseCommit).withLogVersion(logVersion).withFs(fs).build();
|
||||
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit);
|
||||
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
|
||||
.ordinal()));
|
||||
HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header);
|
||||
writer = writer.appendBlock(rollbackBlock);
|
||||
return writer;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReader() throws Exception {
|
||||
testReader(true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNonPartitionedReader() throws Exception {
|
||||
testReader(false);
|
||||
}
|
||||
|
||||
public void testReader(boolean partitioned) throws Exception {
|
||||
// initial commit
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String baseInstant = "100";
|
||||
File partitionDir =
|
||||
partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, baseInstant)
|
||||
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
|
||||
InputFormatTestUtil.commit(basePath, baseInstant);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
|
||||
List<Pair<String, Integer>> logVersionsWithAction = new ArrayList<>();
|
||||
logVersionsWithAction.add(Pair.of(HoodieTimeline.DELTA_COMMIT_ACTION, 1));
|
||||
logVersionsWithAction.add(Pair.of(HoodieTimeline.DELTA_COMMIT_ACTION, 2));
|
||||
// TODO: HUDI-154 Once Hive 2.x PR (PR-674) is merged, enable this change
|
||||
// logVersionsWithAction.add(Pair.of(HoodieTimeline.ROLLBACK_ACTION, 3));
|
||||
FileSlice fileSlice = new FileSlice(partitioned ? FSUtils.getRelativePartitionPath(new Path(
|
||||
basePath.getRoot().getAbsolutePath()), new Path(partitionDir.getAbsolutePath())) : "default",
|
||||
baseInstant, "fileid0");
|
||||
logVersionsWithAction.stream().forEach(logVersionWithAction -> {
|
||||
try {
|
||||
// update files or generate new log file
|
||||
int logVersion = logVersionWithAction.getRight();
|
||||
String action = logVersionWithAction.getKey();
|
||||
int baseInstantTs = Integer.parseInt(baseInstant);
|
||||
String instantTime = String.valueOf(baseInstantTs + logVersion);
|
||||
String latestInstant = action.equals(HoodieTimeline.ROLLBACK_ACTION)
|
||||
? String.valueOf(baseInstantTs + logVersion - 2) : instantTime;
|
||||
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
if (action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
|
||||
writer = writeRollback(partitionDir, schema, "fileid0", baseInstant,
|
||||
instantTime, String.valueOf(baseInstantTs + logVersion - 1), logVersion);
|
||||
} else {
|
||||
writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", baseInstant,
|
||||
instantTime, 100, 0, logVersion);
|
||||
}
|
||||
long size = writer.getCurrentSize();
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
|
||||
//create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
fileSlice.addLogFile(writer.getLogFile());
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1,
|
||||
jobConf), basePath.getRoot().getPath(),
|
||||
fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).map(h -> h.getPath().toString())
|
||||
.collect(Collectors.toList()), instantTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
JobConf jobConf = new JobConf();
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||
if (partitioned) {
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
}
|
||||
|
||||
//validate record reader compaction
|
||||
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
|
||||
//use reader to read base Parquet File and log file, merge in flight and return latest commit
|
||||
//here all 100 records should be updated, see above
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
while (recordReader.next(key, value)) {
|
||||
Writable[] values = value.get();
|
||||
//check if the record written is with latest commit, here "101"
|
||||
Assert.assertEquals(latestInstant, values[0].toString());
|
||||
key = recordReader.createKey();
|
||||
value = recordReader.createValue();
|
||||
}
|
||||
} catch (Exception ioe) {
|
||||
throw new HoodieException(ioe.getMessage(), ioe);
|
||||
}
|
||||
});
|
||||
|
||||
// Add Rollback last version to next log-file
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnMergedReader() throws Exception {
|
||||
// initial commit
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String commitTime = "100";
|
||||
final int numRecords = 1000;
|
||||
final int firstBatchLastRecordKey = numRecords - 1;
|
||||
final int secondBatchLastRecordKey = 2 * numRecords - 1;
|
||||
File partitionDir = InputFormatTestUtil
|
||||
.prepareParquetDataset(basePath, schema, 1, numRecords, commitTime);
|
||||
InputFormatTestUtil.commit(basePath, commitTime);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
|
||||
// insert new records to log file
|
||||
String newCommitTime = "101";
|
||||
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, numRecords, numRecords, 0);
|
||||
long size = writer.getCurrentSize();
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
|
||||
//create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
String logFilePath = writer.getLogFile().getPath().toString();
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1,
|
||||
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
JobConf jobConf = new JobConf();
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
// Enable merge skipping.
|
||||
jobConf.set("hoodie.realtime.merge.skip", "true");
|
||||
|
||||
//validate unmerged record reader
|
||||
RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader);
|
||||
|
||||
//use reader to read base Parquet File and log file
|
||||
//here all records should be present. Also ensure log records are in order.
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
int numRecordsAtCommit1 = 0;
|
||||
int numRecordsAtCommit2 = 0;
|
||||
Set<Integer> seenKeys = new HashSet<>();
|
||||
Integer lastSeenKeyFromLog = firstBatchLastRecordKey;
|
||||
while (recordReader.next(key, value)) {
|
||||
Writable[] values = value.get();
|
||||
String gotCommit = values[0].toString();
|
||||
String keyStr = values[2].toString();
|
||||
Integer gotKey = Integer.parseInt(keyStr.substring("key".length()));
|
||||
if (gotCommit.equals(newCommitTime)) {
|
||||
numRecordsAtCommit2++;
|
||||
Assert.assertTrue(gotKey > firstBatchLastRecordKey);
|
||||
Assert.assertTrue(gotKey <= secondBatchLastRecordKey);
|
||||
Assert.assertEquals(gotKey.intValue(), lastSeenKeyFromLog + 1);
|
||||
lastSeenKeyFromLog++;
|
||||
} else {
|
||||
numRecordsAtCommit1++;
|
||||
Assert.assertTrue(gotKey >= 0);
|
||||
Assert.assertTrue(gotKey <= firstBatchLastRecordKey);
|
||||
}
|
||||
// Ensure unique key
|
||||
Assert.assertFalse(seenKeys.contains(gotKey));
|
||||
seenKeys.add(gotKey);
|
||||
key = recordReader.createKey();
|
||||
value = recordReader.createValue();
|
||||
}
|
||||
Assert.assertEquals(numRecords, numRecordsAtCommit1);
|
||||
Assert.assertEquals(numRecords, numRecordsAtCommit2);
|
||||
Assert.assertEquals(2 * numRecords, seenKeys.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReaderWithNestedAndComplexSchema() throws Exception {
|
||||
// initial commit
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String commitTime = "100";
|
||||
int numberOfRecords = 100;
|
||||
int numberOfLogRecords = numberOfRecords / 2;
|
||||
File partitionDir = InputFormatTestUtil
|
||||
.prepareParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
|
||||
InputFormatTestUtil.commit(basePath, commitTime);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
|
||||
// update files or generate new log file
|
||||
String newCommitTime = "101";
|
||||
HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, numberOfLogRecords);
|
||||
long size = writer.getCurrentSize();
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
|
||||
|
||||
//create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
String logFilePath = writer.getLogFile().getPath().toString();
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1,
|
||||
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
JobConf jobConf = new JobConf();
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
|
||||
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
|
||||
String positions = fields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
|
||||
// validate record reader compaction
|
||||
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
|
||||
// use reader to read base Parquet File and log file, merge in flight and return latest commit
|
||||
// here the first 50 records should be updated, see above
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
int numRecordsRead = 0;
|
||||
while (recordReader.next(key, value)) {
|
||||
int currentRecordNo = numRecordsRead;
|
||||
++numRecordsRead;
|
||||
Writable[] values = value.get();
|
||||
String recordCommitTime;
|
||||
//check if the record written is with latest commit, here "101"
|
||||
if (numRecordsRead > numberOfLogRecords) {
|
||||
recordCommitTime = commitTime;
|
||||
} else {
|
||||
recordCommitTime = newCommitTime;
|
||||
}
|
||||
String recordCommitTimeSuffix = "@" + recordCommitTime;
|
||||
|
||||
Assert.assertEquals(values[0].toString(), recordCommitTime);
|
||||
key = recordReader.createKey();
|
||||
value = recordReader.createValue();
|
||||
|
||||
// Assert type STRING
|
||||
Assert.assertEquals("test value for field: field1", values[5].toString(),
|
||||
"field" + currentRecordNo);
|
||||
Assert.assertEquals("test value for field: field2", values[6].toString(),
|
||||
"field" + currentRecordNo + recordCommitTimeSuffix);
|
||||
Assert.assertEquals("test value for field: name", values[7].toString(),
|
||||
"name" + currentRecordNo);
|
||||
|
||||
// Assert type INT
|
||||
IntWritable intWritable = (IntWritable) values[8];
|
||||
Assert.assertEquals("test value for field: favoriteIntNumber", intWritable.get(),
|
||||
currentRecordNo + recordCommitTime.hashCode());
|
||||
|
||||
// Assert type LONG
|
||||
LongWritable longWritable = (LongWritable) values[9];
|
||||
Assert.assertEquals("test value for field: favoriteNumber", longWritable.get(),
|
||||
currentRecordNo + recordCommitTime.hashCode());
|
||||
|
||||
// Assert type FLOAT
|
||||
FloatWritable floatWritable = (FloatWritable) values[10];
|
||||
Assert.assertEquals("test value for field: favoriteFloatNumber", floatWritable.get(),
|
||||
(float) ((currentRecordNo + recordCommitTime.hashCode()) / 1024.0), 0);
|
||||
|
||||
// Assert type DOUBLE
|
||||
DoubleWritable doubleWritable = (DoubleWritable) values[11];
|
||||
Assert.assertEquals("test value for field: favoriteDoubleNumber", doubleWritable.get(),
|
||||
(currentRecordNo + recordCommitTime.hashCode()) / 1024.0, 0);
|
||||
|
||||
// Assert type MAP
|
||||
ArrayWritable mapItem = (ArrayWritable) values[12];
|
||||
Writable mapItemValue1 = mapItem.get()[0];
|
||||
Writable mapItemValue2 = mapItem.get()[1];
|
||||
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get()[0].toString(),
|
||||
"mapItem1");
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get()[0].toString(),
|
||||
"mapItem2");
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue1).get().length, 2);
|
||||
Assert.assertEquals("test value for field: tags", ((ArrayWritable) mapItemValue2).get().length, 2);
|
||||
Writable mapItemValue1value = ((ArrayWritable) mapItemValue1).get()[1];
|
||||
Writable mapItemValue2value = ((ArrayWritable) mapItemValue2).get()[1];
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item1",
|
||||
((ArrayWritable) mapItemValue1value).get()[0].toString(), "item" + currentRecordNo);
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item1",
|
||||
((ArrayWritable) mapItemValue2value).get()[0].toString(), "item2" + currentRecordNo);
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item2",
|
||||
((ArrayWritable) mapItemValue1value).get()[1].toString(),
|
||||
"item" + currentRecordNo + recordCommitTimeSuffix);
|
||||
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item2",
|
||||
((ArrayWritable) mapItemValue2value).get()[1].toString(),
|
||||
"item2" + currentRecordNo + recordCommitTimeSuffix);
|
||||
|
||||
// Assert type RECORD
|
||||
ArrayWritable recordItem = (ArrayWritable) values[13];
|
||||
Writable[] nestedRecord = recordItem.get();
|
||||
Assert.assertEquals("test value for field: testNestedRecord.isAdmin",
|
||||
((BooleanWritable) nestedRecord[0]).get(), false);
|
||||
Assert
|
||||
.assertEquals("test value for field: testNestedRecord.userId", nestedRecord[1].toString(),
|
||||
"UserId" + currentRecordNo + recordCommitTimeSuffix);
|
||||
|
||||
// Assert type ARRAY
|
||||
ArrayWritable arrayValue = (ArrayWritable) values[14];
|
||||
Writable[] arrayValues = arrayValue.get();
|
||||
for (int i = 0; i < arrayValues.length; i++) {
|
||||
Assert.assertEquals("test value for field: stringArray", "stringArray" + i + recordCommitTimeSuffix,
|
||||
arrayValues[i].toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaEvolutionAndRollbackBlockInLastLogFile() throws Exception {
|
||||
// initial commit
|
||||
List<String> logFilePaths = new ArrayList<>();
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
|
||||
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
|
||||
HoodieTableType.MERGE_ON_READ);
|
||||
String commitTime = "100";
|
||||
int numberOfRecords = 100;
|
||||
int numberOfLogRecords = numberOfRecords / 2;
|
||||
File partitionDir = InputFormatTestUtil
|
||||
.prepareSimpleParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
|
||||
InputFormatTestUtil.commit(basePath, commitTime);
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
List<Field> firstSchemaFields = schema.getFields();
|
||||
|
||||
// update files and generate new log file but don't commit
|
||||
schema = SchemaTestUtil.getComplexEvolvedSchema();
|
||||
String newCommitTime = "101";
|
||||
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, numberOfLogRecords, 0, 1);
|
||||
long size = writer.getCurrentSize();
|
||||
logFilePaths.add(writer.getLogFile().getPath().toString());
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
|
||||
// write rollback for the previous block in new log file version
|
||||
newCommitTime = "102";
|
||||
writer = writeRollbackBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
|
||||
newCommitTime, "101", 1);
|
||||
logFilePaths.add(writer.getLogFile().getPath().toString());
|
||||
writer.close();
|
||||
assertTrue("block - size should be > 0", size > 0);
|
||||
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
|
||||
|
||||
//create a split with baseFile (parquet file written earlier) and new log file(s)
|
||||
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
|
||||
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1,
|
||||
jobConf), basePath.getRoot().getPath(), logFilePaths, newCommitTime);
|
||||
|
||||
//create a RecordReader to be used by HoodieRealtimeRecordReader
|
||||
RecordReader<NullWritable, ArrayWritable> reader =
|
||||
new MapredParquetInputFormat().getRecordReader(
|
||||
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
|
||||
jobConf, null);
|
||||
JobConf jobConf = new JobConf();
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
|
||||
assert (firstSchemaFields.containsAll(fields) == false);
|
||||
|
||||
// Try to read all the fields passed by the new schema
|
||||
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
|
||||
String positions = fields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
|
||||
HoodieRealtimeRecordReader recordReader = null;
|
||||
try {
|
||||
// validate record reader compaction
|
||||
recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
throw new RuntimeException("should've failed the previous line");
|
||||
} catch (HoodieException e) {
|
||||
// expected, field not found since the data written with the evolved schema was rolled back
|
||||
}
|
||||
|
||||
// Try to read all the fields passed by the new schema
|
||||
names = firstSchemaFields.stream().map(f -> f.name()).collect(Collectors.joining(","));
|
||||
positions = firstSchemaFields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
// This time read only the fields which are part of parquet
|
||||
recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
|
||||
// use reader to read base Parquet File and log file
|
||||
NullWritable key = recordReader.createKey();
|
||||
ArrayWritable value = recordReader.createValue();
|
||||
while (recordReader.next(key, value)) {
|
||||
// keep reading
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,23 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, A1
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
25
hudi-hadoop-mr/src/test/resources/log4j-surefire.properties
Normal file
25
hudi-hadoop-mr/src/test/resources/log4j-surefire.properties
Normal file
@@ -0,0 +1,25 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, A1
|
||||
log4j.category.com.uber=INFO
|
||||
log4j.category.org.apache.parquet.hadoop=WARN
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
17
hudi-hadoop-mr/src/test/resources/sample1.avsc
Normal file
17
hudi-hadoop-mr/src/test/resources/sample1.avsc
Normal file
@@ -0,0 +1,17 @@
|
||||
{
|
||||
"type" : "record",
|
||||
"name" : "testRecord",
|
||||
"fields" : [ {
|
||||
"name" : "field1",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "field2",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "_hoodie_commit_time",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "_hoodie_commit_seqno",
|
||||
"type" : "string"
|
||||
}]
|
||||
}
|
||||
Reference in New Issue
Block a user