1
0

CodeStyle formatting to conform to basic Checkstyle rules.

The code-style rules follow google style with some changes:

1. Increase line length from 100 to 120
2. Disable JavaDoc related checkstyles as this needs more manual work.

Both source and test code are checked for code-style
This commit is contained in:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -16,7 +16,6 @@
package com.uber.hoodie.hadoop;
import static org.junit.Assert.assertTrue;
import java.lang.annotation.Annotation;

View File

@@ -82,17 +82,17 @@ public class HoodieInputFormatTest {
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);
"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);
"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);
"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
@@ -108,8 +108,8 @@ public class HoodieInputFormatTest {
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(
"We should exclude commit 100 when returning incremental pull with start commit time as 100",
0, files.length);
"We should exclude commit 100 when returning incremental pull with start commit time as "
+ "100", 0, files.length);
}
@Test
@@ -146,8 +146,8 @@ public class HoodieInputFormatTest {
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);
"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",
@@ -159,23 +159,18 @@ public class HoodieInputFormatTest {
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);
"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
@@ -183,8 +178,7 @@ public class HoodieInputFormatTest {
// initial commit
Schema schema = InputFormatTestUtil.readSchema("/sample1.avro");
String commit1 = "20160628071126";
File partitionDir =
InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
InputFormatTestUtil.commit(basePath, commit1);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -212,19 +206,20 @@ public class HoodieInputFormatTest {
commit2, 2, 10);
}
private void ensureRecordsInCommit(String msg, String commit,
int expectedNumberOfRecordsInCommit, int totalExpected) throws IOException {
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<Void, ArrayWritable>
recordReader = inputFormat.getRecordReader(split, jobConf, null);
RecordReader<Void, ArrayWritable> recordReader = inputFormat
.getRecordReader(split, jobConf, null);
Void 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]
// 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++;

View File

@@ -42,16 +42,15 @@ public class InputFormatTestUtil {
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, 1, "fileid" + i));
File dataFile = new File(partitionPath,
FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i));
dataFile.createNewFile();
}
return partitionPath;
}
public static void simulateUpdates(File directory, final String originalCommit,
int numberOfFilesUpdated,
String newCommit, boolean randomize) throws IOException {
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) {
@@ -62,8 +61,8 @@ public class InputFormatTestUtil {
if (randomize) {
Collections.shuffle(dataFiles);
}
List<File> toUpdateList =
dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
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, 1, fileId));
@@ -78,8 +77,8 @@ public class InputFormatTestUtil {
public static void setupIncremental(JobConf jobConf, String startCommit,
int numberOfCommitsToPull) {
String modePropertyName = String.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN,
HoodieTestUtils.RAW_TRIPS_TEST_NAME);
String modePropertyName = String
.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE);
String startCommitTimestampName = String
@@ -96,19 +95,16 @@ public class InputFormatTestUtil {
}
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema,
int numberOfFiles, int numberOfRecords,
String commitNumber) throws IOException {
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, 1, "fileid" + i);
File dataFile =
new File(partitionPath, fileId);
File dataFile = new File(partitionPath, fileId);
// dataFile.createNewFile();
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
try {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber,
fileId)) {
@@ -132,8 +128,7 @@ public class InputFormatTestUtil {
}
public static void simulateParquetUpdates(File directory, Schema schema, String originalCommit,
int totalNumberOfRecords, int numberOfRecordsToUpdate,
String newCommit) throws IOException {
int totalNumberOfRecords, int numberOfRecordsToUpdate, String newCommit) throws IOException {
File fileToUpdate = directory.listFiles(new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
@@ -145,8 +140,8 @@ public class InputFormatTestUtil {
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
try {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords,
originalCommit, fileId)) {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit,
fileId)) {
if (numberOfRecordsToUpdate > 0) {
// update this record
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, newCommit);

View File

@@ -13,6 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.hadoop;
import static org.junit.Assert.assertFalse;

View File

@@ -18,7 +18,6 @@
package com.uber.hoodie.hadoop.realtime;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Maps;
@@ -85,9 +84,7 @@ public class HoodieRealtimeRecordReaderTest {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.overBaseCommit(baseCommit)
.withFs(fs)
.build();
.overBaseCommit(baseCommit).withFs(fs).build();
List<IndexedRecord> records = new ArrayList<>();
for (int i = 0; i < numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
@@ -106,8 +103,8 @@ public class HoodieRealtimeRecordReaderTest {
public void testReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils
.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
File partitionDir = InputFormatTestUtil
.prepareParquetDataset(basePath, schema, 1, 100, commitTime);
@@ -125,15 +122,15 @@ public class HoodieRealtimeRecordReaderTest {
//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_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(),
Arrays.asList(logFilePath), newCommitTime);
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()),
(String[]) null), jobConf, null);
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(","));
@@ -164,8 +161,8 @@ public class HoodieRealtimeRecordReaderTest {
public void testReaderWithNestedAndComplexSchema() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
HoodieTestUtils
.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.initTableType(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
@@ -185,15 +182,15 @@ public class HoodieRealtimeRecordReaderTest {
//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_" + commitTime + ".parquet"), 0, 1, jobConf), basePath.getRoot().getPath(),
Arrays.asList(logFilePath), newCommitTime);
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()),
(String[]) null), jobConf, null);
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();