1
0

[HUDI-583] Code Cleanup, remove redundant code, and other changes (#1237)

This commit is contained in:
Suneel Marthi
2020-02-02 11:03:44 +01:00
committed by GitHub
parent f27c7a16c6
commit 5b7bb142dc
69 changed files with 447 additions and 582 deletions

View File

@@ -869,12 +869,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
LOG.info("Generate a new instant time " + instantTime);
HoodieTableMetaClient metaClient = createMetaClient(true);
// if there are pending compactions, their instantTime must not be greater than that of this instant time
metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending -> {
metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending ->
Preconditions.checkArgument(
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), instantTime, HoodieTimeline.LESSER),
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
+ latestPending + ", Ingesting at " + instantTime);
});
+ latestPending + ", Ingesting at " + instantTime));
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = table.getMetaClient().getCommitActionType();

View File

@@ -49,15 +49,13 @@ public class MercifulJsonConverter {
* Build type processor map for each avro type.
*/
private static Map<Schema.Type, JsonToAvroFieldProcessor> getFieldTypeProcessors() {
Map<Schema.Type, JsonToAvroFieldProcessor> processorMap =
new ImmutableMap.Builder<Schema.Type, JsonToAvroFieldProcessor>().put(Type.STRING, generateStringTypeHandler())
return new ImmutableMap.Builder<Type, JsonToAvroFieldProcessor>().put(Type.STRING, generateStringTypeHandler())
.put(Type.BOOLEAN, generateBooleanTypeHandler()).put(Type.DOUBLE, generateDoubleTypeHandler())
.put(Type.FLOAT, generateFloatTypeHandler()).put(Type.INT, generateIntTypeHandler())
.put(Type.LONG, generateLongTypeHandler()).put(Type.ARRAY, generateArrayTypeHandler())
.put(Type.RECORD, generateRecordTypeHandler()).put(Type.ENUM, generateEnumTypeHandler())
.put(Type.MAP, generateMapTypeHandler()).put(Type.BYTES, generateBytesTypeHandler())
.put(Type.FIXED, generateFixedTypeHandler()).build();
return processorMap;
}
/**
@@ -286,7 +284,7 @@ public class MercifulJsonConverter {
public Pair<Boolean, Object> convert(Object value, String name, Schema schema)
throws HoodieJsonToAvroConversionException {
Schema elementSchema = schema.getElementType();
List listRes = new ArrayList();
List<Object> listRes = new ArrayList<>();
for (Object v : (List) value) {
listRes.add(convertJsonToAvroField(v, name, elementSchema));
}
@@ -301,7 +299,7 @@ public class MercifulJsonConverter {
public Pair<Boolean, Object> convert(Object value, String name, Schema schema)
throws HoodieJsonToAvroConversionException {
Schema valueSchema = schema.getValueType();
Map<String, Object> mapRes = new HashMap<String, Object>();
Map<String, Object> mapRes = new HashMap<>();
for (Map.Entry<String, Object> v : ((Map<String, Object>) value).entrySet()) {
mapRes.put(v.getKey(), convertJsonToAvroField(v.getValue(), name, valueSchema));
}

View File

@@ -98,8 +98,7 @@ class InternalDynamicBloomFilter extends InternalFilter {
@Override
public void and(InternalFilter filter) {
if (filter == null
|| !(filter instanceof InternalDynamicBloomFilter)
if (!(filter instanceof InternalDynamicBloomFilter)
|| filter.vectorSize != this.vectorSize
|| filter.nbHash != this.nbHash) {
throw new IllegalArgumentException("filters cannot be and-ed");
@@ -122,8 +121,8 @@ class InternalDynamicBloomFilter extends InternalFilter {
return true;
}
for (int i = 0; i < matrix.length; i++) {
if (matrix[i].membershipTest(key)) {
for (BloomFilter bloomFilter : matrix) {
if (bloomFilter.membershipTest(key)) {
return true;
}
}
@@ -133,15 +132,14 @@ class InternalDynamicBloomFilter extends InternalFilter {
@Override
public void not() {
for (int i = 0; i < matrix.length; i++) {
matrix[i].not();
for (BloomFilter bloomFilter : matrix) {
bloomFilter.not();
}
}
@Override
public void or(InternalFilter filter) {
if (filter == null
|| !(filter instanceof InternalDynamicBloomFilter)
if (!(filter instanceof InternalDynamicBloomFilter)
|| filter.vectorSize != this.vectorSize
|| filter.nbHash != this.nbHash) {
throw new IllegalArgumentException("filters cannot be or-ed");
@@ -159,8 +157,7 @@ class InternalDynamicBloomFilter extends InternalFilter {
@Override
public void xor(InternalFilter filter) {
if (filter == null
|| !(filter instanceof InternalDynamicBloomFilter)
if (!(filter instanceof InternalDynamicBloomFilter)
|| filter.vectorSize != this.vectorSize
|| filter.nbHash != this.nbHash) {
throw new IllegalArgumentException("filters cannot be xor-ed");
@@ -180,8 +177,8 @@ class InternalDynamicBloomFilter extends InternalFilter {
public String toString() {
StringBuilder res = new StringBuilder();
for (int i = 0; i < matrix.length; i++) {
res.append(matrix[i]);
for (BloomFilter bloomFilter : matrix) {
res.append(bloomFilter);
res.append(Character.LINE_SEPARATOR);
}
return res.toString();
@@ -195,8 +192,8 @@ class InternalDynamicBloomFilter extends InternalFilter {
out.writeInt(nr);
out.writeInt(currentNbRecord);
out.writeInt(matrix.length);
for (int i = 0; i < matrix.length; i++) {
matrix[i].write(out);
for (BloomFilter bloomFilter : matrix) {
bloomFilter.write(out);
}
}
@@ -217,13 +214,9 @@ class InternalDynamicBloomFilter extends InternalFilter {
* Adds a new row to <i>this</i> dynamic Bloom filter.
*/
private void addRow() {
org.apache.hadoop.util.bloom.BloomFilter[] tmp = new org.apache.hadoop.util.bloom.BloomFilter[matrix.length + 1];
for (int i = 0; i < matrix.length; i++) {
tmp[i] = matrix[i];
}
tmp[tmp.length - 1] = new org.apache.hadoop.util.bloom.BloomFilter(vectorSize, nbHash, hashType);
BloomFilter[] tmp = new BloomFilter[matrix.length + 1];
System.arraycopy(matrix, 0, tmp, 0, matrix.length);
tmp[tmp.length - 1] = new BloomFilter(vectorSize, nbHash, hashType);
matrix = tmp;
}

View File

@@ -41,7 +41,7 @@ import java.nio.charset.StandardCharsets;
public class SimpleBloomFilter implements BloomFilter {
private org.apache.hadoop.util.bloom.BloomFilter filter = null;
private org.apache.hadoop.util.bloom.BloomFilter filter;
/**
* Create a new Bloom filter with the given configurations.
@@ -114,8 +114,7 @@ public class SimpleBloomFilter implements BloomFilter {
filter.write(os);
}
private void readObject(ObjectInputStream is)
throws IOException, ClassNotFoundException {
private void readObject(ObjectInputStream is) throws IOException {
filter = new org.apache.hadoop.util.bloom.BloomFilter();
filter.readFields(is);
}

View File

@@ -114,7 +114,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
}
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
public void initialize(URI uri, Configuration conf) {
// Get the default filesystem to decorate
Path path = new Path(uri);
// Remove 'hoodie-' prefix from path

View File

@@ -66,8 +66,8 @@ public abstract class TimelineLayout implements Serializable {
@Override
public Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> instantStream) {
return instantStream.collect(Collectors.groupingBy(instant -> Pair.of(instant.getTimestamp(),
HoodieInstant.getComparableAction(instant.getAction())))).entrySet().stream()
.map(e -> e.getValue().stream().reduce((x, y) -> {
HoodieInstant.getComparableAction(instant.getAction())))).values().stream()
.map(hoodieInstants -> hoodieInstants.stream().reduce((x, y) -> {
// Pick the one with the highest state
if (x.getState().compareTo(y.getState()) >= 0) {
return x;

View File

@@ -64,10 +64,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION));
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
@@ -79,7 +79,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
*/
public static String createNewInstantTime() {
return lastInstantTime.updateAndGet((oldVal) -> {
String newCommitTime = null;
String newCommitTime;
do {
newCommitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
} while (HoodieTimeline.compareTimestamps(newCommitTime, oldVal, LESSER_OR_EQUAL));
@@ -255,7 +255,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public void deleteCompactionRequested(HoodieInstant instant) {
Preconditions.checkArgument(instant.isRequested());
Preconditions.checkArgument(instant.getAction() == HoodieTimeline.COMPACTION_ACTION);
Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
deleteInstantFile(instant);
}

View File

@@ -65,7 +65,7 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
final MessageDigest md;
try {
md = MessageDigest.getInstance(HASHING_ALGORITHM);
this.instants.stream().forEach(i -> md
this.instants.forEach(i -> md
.update(StringUtils.joinUsingDelim("_", i.getTimestamp(), i.getAction(), i.getState().name()).getBytes()));
} catch (NoSuchAlgorithmException nse) {
throw new HoodieException(nse);

View File

@@ -45,13 +45,13 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION).build();
public static final Comparator<HoodieInstant> ACTION_COMPARATOR =
Comparator.<HoodieInstant, String>comparing(instant -> getComparableAction(instant.getAction()));
Comparator.comparing(instant -> getComparableAction(instant.getAction()));
public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
.thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
public static final String getComparableAction(String action) {
return COMPARABLE_ACTIONS.containsKey(action) ? COMPARABLE_ACTIONS.get(action) : action;
public static String getComparableAction(String action) {
return COMPARABLE_ACTIONS.getOrDefault(action, action);
}
/**

View File

@@ -95,7 +95,7 @@ public class FileSystemViewManager {
* Closes all views opened.
*/
public void close() {
this.globalViewMap.values().stream().forEach(v -> v.close());
this.globalViewMap.values().forEach(SyncableFileSystemView::close);
this.globalViewMap.clear();
}
@@ -196,7 +196,7 @@ public class FileSystemViewManager {
return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> {
RemoteHoodieTableFileSystemView remoteFileSystemView =
createRemoteFileSystemView(conf, viewConfig, new HoodieTableMetaClient(conf.newCopy(), basePath));
SyncableFileSystemView secondaryView = null;
SyncableFileSystemView secondaryView;
switch (viewConfig.getSecondaryStorageType()) {
case MEMORY:
secondaryView = createInMemoryFileSystemView(conf, viewConfig, basePath);

View File

@@ -69,7 +69,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
}
public boolean isIncrementalTimelineSyncEnabled() {
return Boolean.valueOf(props.getProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE));
return Boolean.parseBoolean(props.getProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE));
}
public String getRemoteViewServerHost() {
@@ -87,10 +87,8 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
public long getMaxMemoryForPendingCompaction() {
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
long reservedForPendingComaction =
new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION)))
return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION)))
.longValue();
return reservedForPendingComaction;
}
public String getBaseStoreDir() {
@@ -113,12 +111,9 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
try (FileReader reader = new FileReader(propertiesFile)) {
props.load(reader);
return this;
} finally {
reader.close();
}
}

View File

@@ -85,7 +85,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
schemaHelper.getAllColumnFamilies().stream().forEach(rocksDB::addColumnFamily);
schemaHelper.getAllColumnFamilies().forEach(rocksDB::addColumnFamily);
super.init(metaClient, visibleActiveTimeline);
LOG.info("Created ROCKSDB based file-system view at " + config.getRocksdbBasePath());
}
@@ -98,30 +98,30 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
rocksDB.writeBatch(batch -> {
operations.forEach(opPair -> {
operations.forEach(opPair ->
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opPair.getValue().getFileGroupId()), opPair);
});
schemaHelper.getKeyForPendingCompactionLookup(opPair.getValue().getFileGroupId()), opPair)
);
LOG.info("Initializing pending compaction operations. Count=" + batch.count());
});
}
@Override
protected void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
rocksDB.writeBatch(batch -> {
rocksDB.writeBatch(batch ->
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(!isPendingCompactionScheduledForFileId(opInstantPair.getValue().getFileGroupId()),
"Duplicate FileGroupId found in pending compaction operations. FgId :"
+ opInstantPair.getValue().getFileGroupId());
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()), opInstantPair);
});
});
})
);
}
@Override
void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
rocksDB.writeBatch(batch -> {
rocksDB.writeBatch(batch ->
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(
getPendingCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null,
@@ -129,8 +129,8 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
+ opInstantPair.getValue().getFileGroupId());
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()));
});
});
})
);
}
@Override
@@ -170,17 +170,16 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
schemaHelper.getPrefixForDataFileViewByPartition(partitionPath));
// Now add them
fileGroups.stream().forEach(fg -> {
rocksDB.writeBatch(batch -> {
fileGroups.forEach(fg ->
rocksDB.writeBatch(batch ->
fg.getAllFileSlicesIncludingInflight().forEach(fs -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
fs.getBaseFile().ifPresent(df -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
df);
});
});
});
});
fs.getBaseFile().ifPresent(df ->
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs), df)
);
})
)
);
// record that partition is loaded.
rocksDB.put(schemaHelper.getColFamilyForStoredPartitions(), lookupKey, Boolean.TRUE);
@@ -194,8 +193,8 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
*/
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
DeltaApplyMode mode) {
rocksDB.writeBatch(batch -> {
deltaFileGroups.stream().forEach(fg -> {
rocksDB.writeBatch(batch ->
deltaFileGroups.forEach(fg ->
fg.getAllRawFileSlices().map(fs -> {
FileSlice oldSlice = getFileSlice(partition, fs.getFileId(), fs.getBaseInstantTime());
if (null == oldSlice) {
@@ -203,10 +202,8 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
} else {
// First remove the file-slice
LOG.info("Removing old Slice in DB. FS=" + oldSlice);
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
schemaHelper.getKeyForSliceView(fg, oldSlice));
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
schemaHelper.getKeyForDataFileView(fg, oldSlice));
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, oldSlice));
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, oldSlice));
Map<String, HoodieLogFile> logFiles = oldSlice.getLogFiles()
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
@@ -218,12 +215,12 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
switch (mode) {
case ADD: {
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
oldSlice.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df));
fs.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df));
oldSlice.getBaseFile().ifPresent(newFileSlice::setBaseFile);
fs.getBaseFile().ifPresent(newFileSlice::setBaseFile);
Map<String, HoodieLogFile> newLogFiles = new HashMap<>(logFiles);
deltaLogFiles.entrySet().stream().filter(e -> !logFiles.containsKey(e.getKey()))
.forEach(p -> newLogFiles.put(p.getKey(), p.getValue()));
newLogFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
newLogFiles.values().forEach(newFileSlice::addLogFile);
LOG.info("Adding back new File Slice after add FS=" + newFileSlice);
return newFileSlice;
}
@@ -231,13 +228,13 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
LOG.info("Removing old File Slice =" + fs);
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
fs.getBaseFile().orElseGet(() -> {
oldSlice.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df));
oldSlice.getBaseFile().ifPresent(newFileSlice::setBaseFile);
return null;
});
deltaLogFiles.keySet().stream().forEach(p -> logFiles.remove(p));
deltaLogFiles.keySet().forEach(logFiles::remove);
// Add remaining log files back
logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
logFiles.values().forEach(newFileSlice::addLogFile);
if (newFileSlice.getBaseFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
LOG.info("Adding back new file-slice after remove FS=" + newFileSlice);
return newFileSlice;
@@ -250,13 +247,12 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
}
}).filter(Objects::nonNull).forEach(fs -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
fs.getBaseFile().ifPresent(df -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
df);
});
});
});
});
fs.getBaseFile().ifPresent(df ->
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs), df)
);
})
)
);
}
@Override
@@ -330,7 +326,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
private FileSlice getFileSlice(String partitionPath, String fileId, String instantTime) {
String key = schemaHelper.getKeyForSliceView(partitionPath, fileId, instantTime);
return rocksDB.<FileSlice>get(schemaHelper.getColFamilyForView(), key);
return rocksDB.get(schemaHelper.getColFamilyForView(), key);
}
@Override

View File

@@ -106,8 +106,6 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
@Override
public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream().flatMap(fg -> {
return ((List<HoodieFileGroup>) fg).stream();
});
return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream().flatMap(fg -> ((List<HoodieFileGroup>) fg).stream());
}
}

View File

@@ -42,7 +42,7 @@ import org.apache.avro.specific.SpecificRecordBase;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -58,7 +58,7 @@ public class AvroUtils {
ImmutableMap.Builder<String, List<HoodieRollbackMetadata>> commitToStatBuilder = ImmutableMap.builder();
for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) {
commitToStatBuilder.put(commitToStat.getKey(),
Arrays.asList(convertRollbackMetadata(startRestoreTime, durationInMs, commits, commitToStat.getValue())));
Collections.singletonList(convertRollbackMetadata(startRestoreTime, durationInMs, commits, commitToStat.getValue())));
}
return new HoodieRestoreMetadata(startRestoreTime, durationInMs.orElseGet(() -> -1L), commits,
commitToStatBuilder.build(), DEFAULT_VERSION);

View File

@@ -1,4 +1,4 @@
/**
/*
* 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
@@ -59,7 +59,7 @@ import java.nio.ByteBuffer;
public final class BufferedRandomAccessFile extends RandomAccessFile {
private static final Logger LOG = Logger.getLogger(BufferedRandomAccessFile.class);
static final int DEFAULT_BUFFER_SIZE = (1 << 16); // 64K buffer
static final int BUFFER_BOUNDARY_MASK = ~(DEFAULT_BUFFER_SIZE - 1);
static final int BUFFER_BOUNDARY_MASK = -DEFAULT_BUFFER_SIZE;
private int capacity;
private ByteBuffer dataBuffer;

View File

@@ -55,10 +55,9 @@ public class CleanerUtils {
}
}
HoodieCleanMetadata metadata = new HoodieCleanMetadata(startCleanTime,
return new HoodieCleanMetadata(startCleanTime,
durationInMs.orElseGet(() -> -1L), totalDeleted, earliestCommitToRetain,
partitionMetadataBuilder.build(), CLEAN_METADATA_VERSION_2);
return metadata;
}
/**

View File

@@ -90,7 +90,7 @@ public class CompactionUtils {
Option<Map<String, String>> extraMetadata,
Option<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder();
extraMetadata.ifPresent(m -> builder.setExtraMetadata(m));
extraMetadata.ifPresent(builder::setExtraMetadata);
builder.setOperations(partitionFileSlicePairs.stream()
.map(pfPair -> buildFromFileSlice(pfPair.getKey(), pfPair.getValue(), metricsCaptureFunction))
@@ -157,16 +157,14 @@ public class CompactionUtils {
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToPendingCompactionWithInstantMap =
new HashMap<>();
pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> {
return getPendingCompactionOperations(instantPlanPair.getKey(), instantPlanPair.getValue());
}).forEach(pair -> {
pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair ->
getPendingCompactionOperations(instantPlanPair.getKey(), instantPlanPair.getValue())).forEach(pair -> {
// Defensive check to ensure a single-fileId does not have more than one pending compaction with different
// file slices. If we find a full duplicate we assume it is caused by eventual nature of the move operation
// on some DFSs.
if (fgIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) {
HoodieCompactionOperation operation = pair.getValue().getValue();
HoodieCompactionOperation anotherOperation =
fgIdToPendingCompactionWithInstantMap.get(pair.getKey()).getValue();
HoodieCompactionOperation anotherOperation = fgIdToPendingCompactionWithInstantMap.get(pair.getKey()).getValue();
if (!operation.equals(anotherOperation)) {
String msg = "Hudi File Id (" + pair.getKey() + ") has more than 1 pending compactions. Instants: "
@@ -183,10 +181,8 @@ public class CompactionUtils {
HoodieInstant instant, HoodieCompactionPlan compactionPlan) {
List<HoodieCompactionOperation> ops = compactionPlan.getOperations();
if (null != ops) {
return ops.stream().map(op -> {
return Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
Pair.of(instant.getTimestamp(), op));
});
return ops.stream().map(op -> Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
Pair.of(instant.getTimestamp(), op)));
} else {
return Stream.empty();
}

View File

@@ -78,12 +78,9 @@ public class ConsistencyGuardConfig extends DefaultHoodieConfig {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
try (FileReader reader = new FileReader(propertiesFile)) {
props.load(reader);
return this;
} finally {
reader.close();
}
}

View File

@@ -70,12 +70,7 @@ public class FSUtils {
private static final long MIN_ROLLBACK_TO_KEEP = 10;
private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_";
private static final PathFilter ALLOW_ALL_FILTER = new PathFilter() {
@Override
public boolean accept(Path file) {
return true;
}
};
private static final PathFilter ALLOW_ALL_FILTER = file -> true;
public static Configuration prepareHadoopConf(Configuration conf) {
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
@@ -93,7 +88,7 @@ public class FSUtils {
public static FileSystem getFs(String path, Configuration conf) {
FileSystem fs;
conf = prepareHadoopConf(conf);
prepareHadoopConf(conf);
try {
fs = new Path(path).getFileSystem(conf);
} catch (IOException e) {
@@ -226,8 +221,7 @@ public class FSUtils {
boolean excludeMetaFolder) throws IOException {
PathFilter pathFilter = excludeMetaFolder ? getExcludeMetaPathFilter() : ALLOW_ALL_FILTER;
FileStatus[] topLevelStatuses = fs.listStatus(new Path(basePathStr));
for (int i = 0; i < topLevelStatuses.length; i++) {
FileStatus child = topLevelStatuses[i];
for (FileStatus child : topLevelStatuses) {
if (child.isFile()) {
boolean success = consumer.apply(child);
if (!success) {
@@ -264,12 +258,7 @@ public class FSUtils {
private static PathFilter getExcludeMetaPathFilter() {
// Avoid listing and including any folders under the metafolder
return (path) -> {
if (path.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME)) {
return false;
}
return true;
};
return (path) -> !path.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME);
}
public static String getInstantTime(String name) {
@@ -396,17 +385,14 @@ public class FSUtils {
public static boolean isLogFile(Path logPath) {
Matcher matcher = LOG_FILE_PATTERN.matcher(logPath.getName());
if (!matcher.find()) {
return false;
}
return true;
return matcher.find();
}
/**
* Get the latest log file written from the list of log files passed in.
*/
public static Option<HoodieLogFile> getLatestLogFile(Stream<HoodieLogFile> logFiles) {
return Option.fromJavaOptional(logFiles.sorted(HoodieLogFile.getReverseLogFileComparator()).findFirst());
return Option.fromJavaOptional(logFiles.min(HoodieLogFile.getReverseLogFileComparator()));
}
/**

View File

@@ -81,7 +81,7 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
public void waitForFilesVisibility(String dirPath, List<String> files, FileVisibility event) throws TimeoutException {
Path dir = new Path(dirPath);
List<String> filesWithoutSchemeAndAuthority =
files.stream().map(f -> Path.getPathWithoutSchemeAndAuthority(new Path(f))).map(p -> p.toString())
files.stream().map(f -> Path.getPathWithoutSchemeAndAuthority(new Path(f))).map(Path::toString)
.collect(Collectors.toList());
retryTillSuccess((retryNum) -> {
@@ -89,7 +89,7 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
LOG.info("Trying " + retryNum);
FileStatus[] entries = fs.listStatus(dir);
List<String> gotFiles = Arrays.stream(entries).map(e -> Path.getPathWithoutSchemeAndAuthority(e.getPath()))
.map(p -> p.toString()).collect(Collectors.toList());
.map(Path::toString).collect(Collectors.toList());
List<String> candidateFiles = new ArrayList<>(filesWithoutSchemeAndAuthority);
boolean altered = candidateFiles.removeAll(gotFiles);

View File

@@ -43,6 +43,7 @@ import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -84,7 +85,7 @@ public class HoodieAvroUtils {
public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException {
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(bytes, reuseDecoder.get());
reuseDecoder.set(decoder);
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema);
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
return reader.read(null, decoder);
}
@@ -141,7 +142,7 @@ public class HoodieAvroUtils {
Schema.Field recordKeyField =
new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
Schema recordKeySchema = Schema.createRecord("HoodieRecordKey", "", "", false);
recordKeySchema.setFields(Arrays.asList(recordKeyField));
recordKeySchema.setFields(Collections.singletonList(recordKeyField));
return recordKeySchema;
}
@@ -166,9 +167,8 @@ public class HoodieAvroUtils {
* @param newFieldNames Null Field names to be added
*/
public static Schema appendNullSchemaFields(Schema schema, List<String> newFieldNames) {
List<Field> newFields = schema.getFields().stream().map(field -> {
return new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue());
}).collect(Collectors.toList());
List<Field> newFields = schema.getFields().stream()
.map(field -> new Field(field.name(), field.schema(), field.doc(), field.defaultValue())).collect(Collectors.toList());
for (String newField : newFieldNames) {
newFields.add(new Schema.Field(newField, METADATA_FIELD_SCHEMA, "", NullNode.getInstance()));
}

View File

@@ -49,7 +49,7 @@ public class LogReaderUtils {
HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
while (reader.hasPrev()) {
HoodieLogBlock block = reader.prev();
if (block instanceof HoodieAvroDataBlock && block != null) {
if (block instanceof HoodieAvroDataBlock) {
HoodieAvroDataBlock lastBlock = (HoodieAvroDataBlock) block;
if (completedTimeline
.containsOrBeforeTimelineStarts(lastBlock.getLogBlockHeader().get(HeaderMetadataType.INSTANT_TIME))) {

View File

@@ -132,7 +132,7 @@ public class ObjectSizeCalculator {
private final Set<Object> alreadyVisited = Sets.newIdentityHashSet();
private final Deque<Object> pending = new ArrayDeque<Object>(16 * 1024);
private final Deque<Object> pending = new ArrayDeque<>(16 * 1024);
private long size;
/**
@@ -268,7 +268,7 @@ public class ObjectSizeCalculator {
public ClassSizeInfo(Class<?> clazz) {
long fieldsSize = 0;
final List<Field> referenceFields = new LinkedList<Field>();
final List<Field> referenceFields = new LinkedList<>();
for (Field f : clazz.getDeclaredFields()) {
if (Modifier.isStatic(f.getModifiers())) {
continue;
@@ -303,9 +303,7 @@ public class ObjectSizeCalculator {
try {
calc.enqueue(f.get(obj));
} catch (IllegalAccessException e) {
final AssertionError ae = new AssertionError("Unexpected denial of access to " + f);
ae.initCause(e);
throw ae;
throw new AssertionError("Unexpected denial of access to " + f, e);
}
}
}

View File

@@ -26,6 +26,7 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.rocksdb.AbstractImmutableNativeReference;
import org.rocksdb.ColumnFamilyDescriptor;
import org.rocksdb.ColumnFamilyHandle;
import org.rocksdb.ColumnFamilyOptions;
@@ -61,13 +62,11 @@ public class RocksDBDAO {
private transient ConcurrentHashMap<String, ColumnFamilyDescriptor> managedDescriptorMap;
private transient RocksDB rocksDB;
private boolean closed = false;
private final String basePath;
private final String rocksDBBasePath;
public RocksDBDAO(String basePath, String rocksDBBasePath) {
this.basePath = basePath;
this.rocksDBBasePath =
String.format("%s/%s/%s", rocksDBBasePath, this.basePath.replace("/", "_"), UUID.randomUUID().toString());
String.format("%s/%s/%s", rocksDBBasePath, basePath.replace("/", "_"), UUID.randomUUID().toString());
init();
}
@@ -153,14 +152,11 @@ public class RocksDBDAO {
* Perform a batch write operation.
*/
public void writeBatch(BatchHandler handler) {
WriteBatch batch = new WriteBatch();
try {
try (WriteBatch batch = new WriteBatch()) {
handler.apply(batch);
getRocksDB().write(new WriteOptions(), batch);
} catch (RocksDBException re) {
throw new HoodieException(re);
} finally {
batch.close();
}
}
@@ -442,9 +438,7 @@ public class RocksDBDAO {
public synchronized void close() {
if (!closed) {
closed = true;
managedHandlesMap.values().forEach(columnFamilyHandle -> {
columnFamilyHandle.close();
});
managedHandlesMap.values().forEach(AbstractImmutableNativeReference::close);
managedHandlesMap.clear();
managedDescriptorMap.clear();
getRocksDB().close();

View File

@@ -41,7 +41,7 @@ public class SerializationUtils {
// Caching kryo serializer to avoid creating kryo instance for every serde operation
private static final ThreadLocal<KryoSerializerInstance> SERIALIZER_REF =
ThreadLocal.withInitial(() -> new KryoSerializerInstance());
ThreadLocal.withInitial(KryoSerializerInstance::new);
// Serialize
// -----------------------------------------------------------------------
@@ -99,7 +99,7 @@ public class SerializationUtils {
kryo.setRegistrationRequired(false);
}
byte[] serialize(Object obj) throws IOException {
byte[] serialize(Object obj) {
kryo.reset();
baos.reset();
Output output = new Output(baos);

View File

@@ -84,11 +84,7 @@ public final class RocksDBBasedMap<K extends Serializable, R extends Serializabl
@Override
public void putAll(Map<? extends K, ? extends R> m) {
getRocksDBDAO().writeBatch(batch -> {
m.entrySet().forEach(entry -> {
getRocksDBDAO().putInBatch(batch, columnFamilyName, entry.getKey(), entry.getValue());
});
});
getRocksDBDAO().writeBatch(batch -> m.forEach((key, value) -> getRocksDBDAO().putInBatch(batch, columnFamilyName, key, value)));
}
private RocksDBDAO getRocksDBDAO() {

View File

@@ -73,7 +73,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
// it holds the root cause of the exception in case either queueing records (consuming from
// inputIterator) fails or
// thread reading records from queue fails.
private final AtomicReference<Exception> hasFailed = new AtomicReference(null);
private final AtomicReference<Exception> hasFailed = new AtomicReference<>(null);
// used for indicating that all the records from queue are read successfully.
private final AtomicBoolean isReadDone = new AtomicBoolean(false);
// used for indicating that all records have been enqueued
@@ -222,7 +222,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
/**
* Puts an empty entry to queue to denote termination.
*/
public void close() throws InterruptedException {
public void close() {
// done queueing records notifying queue-reader.
isWriteDone.set(true);
}

View File

@@ -70,6 +70,7 @@ import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Calendar;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.Iterator;
@@ -125,7 +126,7 @@ public class HoodieTestUtils {
return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
}
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
public static void createCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime : commitTimes) {
new File(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
@@ -139,11 +140,11 @@ public class HoodieTestUtils {
}
}
public static final void createMetadataFolder(String basePath) {
public static void createMetadataFolder(String basePath) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
}
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
public static void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime : commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
@@ -153,7 +154,7 @@ public class HoodieTestUtils {
}
}
public static final void createPendingCleanFiles(HoodieTableMetaClient metaClient, String... commitTimes) {
public static void createPendingCleanFiles(HoodieTableMetaClient metaClient, String... commitTimes) {
for (String commitTime : commitTimes) {
Arrays.asList(HoodieTimeline.makeRequestedCleanerFileName(commitTime),
HoodieTimeline.makeInflightCleanerFileName(commitTime)).forEach(f -> {
@@ -180,19 +181,19 @@ public class HoodieTestUtils {
}
}
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime)
public static String createNewDataFile(String basePath, String partitionPath, String commitTime)
throws IOException {
String fileID = UUID.randomUUID().toString();
return createDataFile(basePath, partitionPath, commitTime, fileID);
}
public static final String createNewMarkerFile(String basePath, String partitionPath, String commitTime)
public static String createNewMarkerFile(String basePath, String partitionPath, String commitTime)
throws IOException {
String fileID = UUID.randomUUID().toString();
return createMarkerFile(basePath, partitionPath, commitTime, fileID);
}
public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID)
public static String createDataFile(String basePath, String partitionPath, String commitTime, String fileID)
throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
new File(folderPath).mkdirs();
@@ -200,7 +201,7 @@ public class HoodieTestUtils {
return fileID;
}
public static final String createMarkerFile(String basePath, String partitionPath, String commitTime, String fileID)
public static String createMarkerFile(String basePath, String partitionPath, String commitTime, String fileID)
throws IOException {
String folderPath =
basePath + "/" + HoodieTableMetaClient.TEMPFOLDER_NAME + "/" + commitTime + "/" + partitionPath + "/";
@@ -210,7 +211,7 @@ public class HoodieTestUtils {
return f.getAbsolutePath();
}
public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime,
public static String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime,
String fileID, Option<Integer> version) throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
boolean makeDir = fs.mkdirs(new Path(folderPath));
@@ -226,7 +227,7 @@ public class HoodieTestUtils {
return fileID;
}
public static final void createCompactionCommitFiles(FileSystem fs, String basePath, String... commitTimes)
public static void createCompactionCommitFiles(FileSystem fs, String basePath, String... commitTimes)
throws IOException {
for (String commitTime : commitTimes) {
boolean createFile = fs.createNewFile(new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
@@ -237,7 +238,7 @@ public class HoodieTestUtils {
}
}
public static final void createCompactionRequest(HoodieTableMetaClient metaClient, String instant,
public static void createCompactionRequest(HoodieTableMetaClient metaClient, String instant,
List<Pair<String, FileSlice>> fileSliceList) throws IOException {
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(fileSliceList, Option.empty(), Option.empty());
HoodieInstant compactionInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant);
@@ -245,47 +246,47 @@ public class HoodieTestUtils {
AvroUtils.serializeCompactionPlan(plan));
}
public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) {
public static String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) {
return basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, DEFAULT_WRITE_TOKEN, fileID);
}
public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID,
public static String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID,
Option<Integer> version) {
return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime,
version.orElse(DEFAULT_LOG_VERSION), HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
}
public static final String getCommitFilePath(String basePath, String commitTime) {
public static String getCommitFilePath(String basePath, String commitTime) {
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION;
}
public static final String getInflightCommitFilePath(String basePath, String commitTime) {
public static String getInflightCommitFilePath(String basePath, String commitTime) {
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
}
public static final String getRequestedCompactionFilePath(String basePath, String commitTime) {
public static String getRequestedCompactionFilePath(String basePath, String commitTime) {
return basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + commitTime
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
}
public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime,
public static boolean doesDataFileExist(String basePath, String partitionPath, String commitTime,
String fileID) {
return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists();
}
public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID,
public static boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID,
Option<Integer> version) {
return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists();
}
public static final boolean doesCommitExist(String basePath, String commitTime) {
public static boolean doesCommitExist(String basePath, String commitTime) {
return new File(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION)
.exists();
}
public static final boolean doesInflightExist(String basePath, String commitTime) {
public static boolean doesInflightExist(String basePath, String commitTime) {
return new File(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.INFLIGHT_EXTENSION)
.exists();
@@ -298,19 +299,16 @@ public class HoodieTestUtils {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(commitFile, true);
try {
try (FSDataOutputStream os = fs.create(commitFile, true)) {
HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], new ArrayList<>(), new ArrayList<>(),
new ArrayList<>(), commitTime);
// Create the clean metadata
HoodieCleanMetadata cleanMetadata =
CleanerUtils.convertCleanMetadata(metaClient, commitTime, Option.of(0L), Arrays.asList(cleanStats));
CleanerUtils.convertCleanMetadata(metaClient, commitTime, Option.of(0L), Collections.singletonList(cleanStats));
// Write empty clean metadata
os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get());
} finally {
os.close();
}
}

View File

@@ -25,7 +25,6 @@ import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import com.google.common.collect.Lists;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile;
@@ -34,6 +33,7 @@ import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
import java.util.stream.Collectors;
import static org.junit.Assert.assertArrayEquals;
@@ -120,7 +120,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness {
HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
assertEquals(Lists.newArrayList(instant1, instant2, instant3),
assertEquals(Arrays.asList(instant1, instant2, instant3),
archivedTimeline.getInstants().collect(Collectors.toList()));
assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.getInstantDetails(instant1).get());

View File

@@ -86,7 +86,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
private FileSystem fs;
private Path partitionPath;
private int bufferSize = 4096;
private Boolean readBlocksLazily = true;
private Boolean readBlocksLazily;
public TestHoodieLogFormat(Boolean readBlocksLazily) {
this.readBlocksLazily = readBlocksLazily;

View File

@@ -209,7 +209,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
* @return
*/
private Stream<FileSlice> getAllRawFileSlices(String partitionPath) {
return fsView.getAllFileGroups(partitionPath).map(group -> group.getAllFileSlicesIncludingInflight())
return fsView.getAllFileGroups(partitionPath).map(HoodieFileGroup::getAllFileSlicesIncludingInflight)
.flatMap(sliceList -> sliceList);
}
@@ -220,7 +220,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
* @return
*/
public Stream<FileSlice> getLatestRawFileSlices(String partitionPath) {
return fsView.getAllFileGroups(partitionPath).map(fileGroup -> fileGroup.getLatestFileSlicesIncludingInflight())
return fsView.getAllFileGroups(partitionPath).map(HoodieFileGroup::getLatestFileSlicesIncludingInflight)
.filter(fileSliceOpt -> fileSliceOpt.isPresent()).map(Option::get);
}
@@ -275,7 +275,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0)));
HoodieCompactionPlan compactionPlan =
CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, Option.empty(), Option.empty());
HoodieInstant compactionInstant = null;
HoodieInstant compactionInstant;
if (isCompactionInFlight) {
// Create a Data-file but this should be skipped by view
new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile();

View File

@@ -62,6 +62,7 @@ import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -115,13 +116,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
unscheduleCompaction(view, "14", "13", "11");
// Add one more delta instant
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("15"), true, "11"));
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("15"), true, "11"));
// Schedule Compaction again
scheduleCompaction(view, "16");
// Run Compaction - This will be the second file-slice
testMultipleWriteSteps(view, Arrays.asList("16"), false, "16", 2);
testMultipleWriteSteps(view, Collections.singletonList("16"), false, "16", 2);
// Run 2 more ingest
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("17", "18"), true, "16", 2));
@@ -130,25 +131,25 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
scheduleCompaction(view, "19");
// Run one more ingestion after pending compaction. THis will be 3rd slice
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("20"), true, "19", 3));
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("20"), true, "19", 3));
// Clean first slice
testCleans(view, Arrays.asList("21"),
testCleans(view, Collections.singletonList("21"),
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "15")).build(),
instantsToFiles, Arrays.asList("11"));
instantsToFiles, Collections.singletonList("11"));
// Add one more ingestion instant. This should be 2nd slice now
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("22"), true, "19", 2));
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("22"), true, "19", 2));
// Restore last ingestion
testRestore(view, Arrays.asList("23"), true, new HashMap<>(), Arrays.asList("22"), "24", false);
testRestore(view, Collections.singletonList("23"), true, new HashMap<>(), Collections.singletonList("22"), "24", false);
// Run one more ingestion. THis is still 2nd slice
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("24"), true, "19", 2));
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("24"), true, "19", 2));
// Finish Compaction
instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("19"), false, "19", 2,
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24"))));
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("19"), false, "19", 2,
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24"))));
}
@Test
@@ -198,13 +199,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
SyncableFileSystemView view1 = getFileSystemView(metaClient);
view1.sync();
Map<String, List<String>> instantsToFiles = null;
Map<String, List<String>> instantsToFiles;
/**
* Case where incremental syncing is catching up on more than one ingestion at a time
*/
// Run 1 ingestion on MOR table (1 delta commits). View1 is now sync up to this point
instantsToFiles = testMultipleWriteSteps(view1, Arrays.asList("11"), true, "11");
instantsToFiles = testMultipleWriteSteps(view1, Collections.singletonList("11"), true, "11");
SyncableFileSystemView view2 =
getFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
@@ -213,7 +214,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("12", "13"), true, "11"));
// Now Sync view1 and add 1 more ingestion. Check if view1 is able to catchup correctly
instantsToFiles.putAll(testMultipleWriteSteps(view1, Arrays.asList("14"), true, "11"));
instantsToFiles.putAll(testMultipleWriteSteps(view1, Collections.singletonList("14"), true, "11"));
view2.sync();
SyncableFileSystemView view3 =
@@ -238,8 +239,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
scheduleCompaction(view2, "16");
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("17", "18"), true, "16", 2));
// Compaction
testMultipleWriteSteps(view2, Arrays.asList("16"), false, "16", 2,
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18")));
testMultipleWriteSteps(view2, Collections.singletonList("16"), false, "16", 2,
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18")));
view1.sync();
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
SyncableFileSystemView view5 =
@@ -249,14 +250,14 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
/**
* Case where a clean happened and then rounds of ingestion and compaction happened
*/
testCleans(view2, Arrays.asList("19"),
testCleans(view2, Collections.singletonList("19"),
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "14")).build(),
instantsToFiles, Arrays.asList("11"));
instantsToFiles, Collections.singletonList("11"));
scheduleCompaction(view2, "20");
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2));
// Compaction
testMultipleWriteSteps(view2, Arrays.asList("20"), false, "20", 2,
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22")));
testMultipleWriteSteps(view2, Collections.singletonList("20"), false, "20", 2,
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22")));
// Run one more round of ingestion
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("23", "24"), true, "20", 2));
view1.sync();
@@ -268,14 +269,14 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
/**
* Case where multiple restores and ingestions happened
*/
testRestore(view2, Arrays.asList("25"), true, new HashMap<>(), Arrays.asList("24"), "29", true);
testRestore(view2, Arrays.asList("26"), true, new HashMap<>(), Arrays.asList("23"), "29", false);
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("27"), true, "20", 2));
testRestore(view2, Collections.singletonList("25"), true, new HashMap<>(), Collections.singletonList("24"), "29", true);
testRestore(view2, Collections.singletonList("26"), true, new HashMap<>(), Collections.singletonList("23"), "29", false);
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("27"), true, "20", 2));
scheduleCompaction(view2, "28");
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("29"), true, "28", 3));
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("29"), true, "28", 3));
// Compaction
testMultipleWriteSteps(view2, Arrays.asList("28"), false, "28", 3,
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29")));
testMultipleWriteSteps(view2, Collections.singletonList("28"), false, "28", 3,
Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29")));
Arrays.asList(view1, view2, view3, view4, view5, view6).forEach(v -> {
v.sync();
@@ -371,8 +372,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
LOG.info("Last Instant is :" + view.getLastInstant().get());
if (isRestore) {
Assert.assertEquals(newRestoreInstants.get(idx), view.getLastInstant().get().getTimestamp());
Assert.assertEquals(isRestore ? HoodieTimeline.RESTORE_ACTION : HoodieTimeline.ROLLBACK_ACTION,
view.getLastInstant().get().getAction());
Assert.assertEquals(HoodieTimeline.RESTORE_ACTION, view.getLastInstant().get().getAction());
}
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
@@ -532,9 +532,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
view.sync();
Assert.assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp());
partitions.forEach(p -> view.getLatestFileSlices(p).forEach(fs -> {
Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime());
}));
partitions.forEach(p -> view.getLatestFileSlices(p).forEach(fs -> Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime())));
}
/**
@@ -618,17 +616,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
final long expTotalFileSlicesPerPartition = fileIdsPerPartition.size() * multiple;
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
if (deltaCommit) {
partitions.forEach(p -> {
view.getLatestFileSlices(p).forEach(f -> {
Assert.assertEquals(baseInstantForDeltaCommit, f.getBaseInstantTime());
});
});
partitions.forEach(p ->
view.getLatestFileSlices(p).forEach(f -> Assert.assertEquals(baseInstantForDeltaCommit, f.getBaseInstantTime()))
);
} else {
partitions.forEach(p -> {
view.getLatestBaseFiles(p).forEach(f -> {
Assert.assertEquals(instant, f.getCommitTime());
});
});
partitions.forEach(p -> view.getLatestBaseFiles(p).forEach(f -> Assert.assertEquals(instant, f.getCommitTime())));
}
metaClient.reloadActiveTimeline();
@@ -704,8 +696,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
String baseInstant) throws IOException {
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> {
return fileIdsPerPartition.stream().map(f -> {
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> fileIdsPerPartition.stream().map(f -> {
try {
File file = new File(basePath + "/" + p + "/"
+ (deltaCommit
@@ -719,8 +710,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
} catch (IOException e) {
throw new HoodieException(e);
}
});
}).collect(Collectors.toList());
})).collect(Collectors.toList());
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));

View File

@@ -229,8 +229,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
String log1Ver1 = makeOldLogFileName("file1", ".log", "1", 1);
String log1base2 = makeOldLogFileName("file1", ".log", "2", 0);
List<HoodieLogFile> logFiles = Stream.of(log1base2, log1Ver1, log1Ver0).map(HoodieLogFile::new)
.collect(Collectors.toList());
logFiles.sort(HoodieLogFile.getLogFileComparator());
.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
assertEquals(log1Ver0, logFiles.get(0).getFileName());
assertEquals(log1Ver1, logFiles.get(1).getFileName());
assertEquals(log1base2, logFiles.get(2).getFileName());
@@ -250,8 +249,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
List<HoodieLogFile> logFiles =
Stream.of(log1Ver1W1, log1base2W0, log1base2W1, log1Ver1W0, log1Ver0W1, log1Ver0W0)
.map(HoodieLogFile::new).collect(Collectors.toList());
logFiles.sort(HoodieLogFile.getLogFileComparator());
.map(HoodieLogFile::new).sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
assertEquals(log1Ver0W0, logFiles.get(0).getFileName());
assertEquals(log1Ver0W1, logFiles.get(1).getFileName());
assertEquals(log1Ver1W0, logFiles.get(2).getFileName());

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.hadoop;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@@ -77,8 +78,7 @@ public class HoodieHiveUtil {
return (!matcher.find() ? null : matcher.group(1));
}
return null;
}).filter(s -> s != null)
.collect(Collectors.toList());
}).filter(Objects::nonNull).collect(Collectors.toList());
if (result == null) {
// Returns an empty list instead of null.
result = new ArrayList<>();

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.hadoop;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
@@ -48,6 +49,7 @@ import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -93,9 +95,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
if (nonHoodiePaths.size() > 0) {
setInputPaths(job, nonHoodiePaths.toArray(new Path[nonHoodiePaths.size()]));
FileStatus[] fileStatuses = super.listStatus(job);
for (FileStatus fileStatus: fileStatuses) {
returns.add(fileStatus);
}
returns.addAll(Arrays.asList(fileStatuses));
}
// process snapshot queries next.
@@ -133,8 +133,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
.getInstants().collect(Collectors.toList());
// Extract partitions touched by the commitsToCheck
Set<String> partitionsToList = new HashSet<>();
for (int i = 0; i < commitsToCheck.size(); i++) {
HoodieInstant commit = commitsToCheck.get(i);
for (HoodieInstant commit : commitsToCheck) {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(),
HoodieCommitMetadata.class);
partitionsToList.addAll(commitMetadata.getPartitionToWriteStats().keySet());
@@ -171,15 +170,14 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
return false;
})
.collect(Collectors.joining(","));
if (incrementalInputPaths == null || incrementalInputPaths.isEmpty()) {
if (StringUtils.isNullOrEmpty(incrementalInputPaths)) {
return null;
}
// Mutate the JobConf to set the input paths to only partitions touched by incremental pull.
setInputPaths(job, incrementalInputPaths);
FileStatus[] fileStatuses = super.listStatus(job);
BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline,
fileStatuses);
List<String> commitsList = commitsToCheck.stream().map(s -> s.getTimestamp()).collect(Collectors.toList());
BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline, fileStatuses);
List<String> commitsList = commitsToCheck.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<HoodieBaseFile> filteredFiles = roView.getLatestBaseFilesInRange(commitsList).collect(Collectors.toList());
List<FileStatus> returns = new ArrayList<>();
for (HoodieBaseFile filteredFile : filteredFiles) {
@@ -200,7 +198,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
* @throws IOException
*/
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatusForSnapshotPaths(
FileStatus[] fileStatuses, Collection<HoodieTableMetaClient> metaClientList) throws IOException {
FileStatus[] fileStatuses, Collection<HoodieTableMetaClient> metaClientList) {
// This assumes the paths for different tables are grouped together
Map<HoodieTableMetaClient, List<FileStatus>> grouped = new HashMap<>();
HoodieTableMetaClient metadata = null;
@@ -231,8 +229,8 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
* Filters data files for a snapshot queried table.
*/
private List<FileStatus> filterFileStatusForSnapshotMode(
HoodieTableMetaClient metadata, List<FileStatus> fileStatuses) throws IOException {
FileStatus[] statuses = fileStatuses.toArray(new FileStatus[fileStatuses.size()]);
HoodieTableMetaClient metadata, List<FileStatus> fileStatuses) {
FileStatus[] statuses = fileStatuses.toArray(new FileStatus[0]);
if (LOG.isDebugEnabled()) {
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
}
@@ -258,7 +256,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
* super.listStatus() and gets back a FileStatus[] 2. Then it creates the HoodieTableMetaClient for the paths listed.
* 3. Generation of splits looks at FileStatus size to create splits, which skips this file
*/
private HoodieBaseFile checkFileStatus(HoodieBaseFile dataFile) throws IOException {
private HoodieBaseFile checkFileStatus(HoodieBaseFile dataFile) {
Path dataPath = dataFile.getFileStatus().getPath();
try {
if (dataFile.getFileSize() == 0) {

View File

@@ -65,11 +65,12 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
@@ -115,7 +116,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
@Override
public Set<Integer> call() throws Exception {
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
Set<Integer> nonCombinablePathIndices = new HashSet<>();
for (int i = 0; i < length; i++) {
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo,
paths[i + start], IOPrepareCache.get().allocatePartitionDescMap());
@@ -356,25 +357,21 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim combine =
new HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim();
InputSplit[] splits = null;
if (combine == null) {
splits = super.getSplits(job, numSplits);
return splits;
}
InputSplit[] splits;
if (combine.getInputPathsShim(job).length == 0) {
throw new IOException("No input paths specified in job");
}
ArrayList<InputSplit> result = new ArrayList<InputSplit>();
ArrayList<InputSplit> result = new ArrayList<>();
// combine splits only from same tables and same partitions. Do not combine splits from multiple
// tables or multiple partitions.
Path[] paths = StringInternUtils.internUriStringsInPathArray(combine.getInputPathsShim(job));
List<Path> inpDirs = new ArrayList<Path>();
List<Path> inpFiles = new ArrayList<Path>();
Map<CombinePathInputFormat, CombineFilter> poolMap = new HashMap<CombinePathInputFormat, CombineFilter>();
Set<Path> poolSet = new HashSet<Path>();
List<Path> inpDirs = new ArrayList<>();
List<Path> inpFiles = new ArrayList<>();
Map<CombinePathInputFormat, CombineFilter> poolMap = new HashMap<>();
Set<Path> poolSet = new HashSet<>();
for (Path path : paths) {
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo, path,
@@ -414,8 +411,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
Path filterPath = path;
// Does a pool exist for this path already
CombineFilter f = null;
List<Operator<? extends OperatorDesc>> opList = null;
CombineFilter f;
List<Operator<? extends OperatorDesc>> opList;
if (!mrwork.isMapperCannotSpanPartns()) {
// if mapper can span partitions, make sure a splits does not contain multiple
@@ -441,7 +438,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// parent directory will be grouped into one pool but not files from different parent
// directories. This guarantees that a split will combine all files in the same partition
// but won't cross multiple partitions if the user has asked so.
if (!path.getFileSystem(job).getFileStatus(path).isDir()) { // path is not directory
if (!path.getFileSystem(job).getFileStatus(path).isDirectory()) { // path is not directory
filterPath = path.getParent();
inpFiles.add(path);
poolSet.add(filterPath);
@@ -452,7 +449,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
// Processing directories
List<CombineFileSplit> iss = new ArrayList<CombineFileSplit>();
List<CombineFileSplit> iss = new ArrayList<>();
if (!mrwork.isMapperCannotSpanPartns()) {
// mapper can span partitions
// combine into as few as one split, subject to the PathFilters set
@@ -496,14 +493,14 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
int numPathPerThread = (int) Math.ceil((double) paths.length / numThreads);
ExecutorService executor = Executors.newFixedThreadPool(numThreads);
List<Future<Set<Integer>>> futureList = new ArrayList<Future<Set<Integer>>>(numThreads);
List<Future<Set<Integer>>> futureList = new ArrayList<>(numThreads);
try {
for (int i = 0; i < numThreads; i++) {
int start = i * numPathPerThread;
int length = i != numThreads - 1 ? numPathPerThread : paths.length - start;
futureList.add(executor.submit(new CheckNonCombinablePathCallable(paths, start, length, job)));
}
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
Set<Integer> nonCombinablePathIndices = new HashSet<>();
for (Future<Set<Integer>> future : futureList) {
nonCombinablePathIndices.addAll(future.get());
}
@@ -522,12 +519,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.GET_SPLITS);
init(job);
ArrayList<InputSplit> result = new ArrayList<InputSplit>();
List<InputSplit> result = new ArrayList<>();
Path[] paths = getInputPaths(job);
List<Path> nonCombinablePaths = new ArrayList<Path>(paths.length / 2);
List<Path> combinablePaths = new ArrayList<Path>(paths.length / 2);
List<Path> nonCombinablePaths = new ArrayList<>(paths.length / 2);
List<Path> combinablePaths = new ArrayList<>(paths.length / 2);
int numThreads = Math.min(MAX_CHECK_NONCOMBINABLE_THREAD_NUM,
(int) Math.ceil((double) paths.length / DEFAULT_NUM_PATH_PER_THREAD));
@@ -561,22 +558,18 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// Process the normal splits
if (nonCombinablePaths.size() > 0) {
FileInputFormat.setInputPaths(job, nonCombinablePaths.toArray(new Path[nonCombinablePaths.size()]));
FileInputFormat.setInputPaths(job, nonCombinablePaths.toArray(new Path[0]));
InputSplit[] splits = super.getSplits(job, numSplits);
for (InputSplit split : splits) {
result.add(split);
}
Collections.addAll(result, splits);
}
// Process the combine splits
if (combinablePaths.size() > 0) {
FileInputFormat.setInputPaths(job, combinablePaths.toArray(new Path[combinablePaths.size()]));
FileInputFormat.setInputPaths(job, combinablePaths.toArray(new Path[0]));
Map<Path, PartitionDesc> pathToPartitionInfo = this.pathToPartitionInfo != null ? this.pathToPartitionInfo
: Utilities.getMapWork(job).getPathToPartitionInfo();
InputSplit[] splits = getCombineSplits(job, numSplits, pathToPartitionInfo);
for (InputSplit split : splits) {
result.add(split);
}
Collections.addAll(result, splits);
}
// Restore the old path information back
@@ -634,8 +627,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
*/
private List<CombineFileSplit> sampleSplits(List<CombineFileSplit> splits) {
HashMap<String, SplitSample> nameToSamples = mrwork.getNameToSplitSample();
List<CombineFileSplit> retLists = new ArrayList<CombineFileSplit>();
Map<String, ArrayList<CombineFileSplit>> aliasToSplitList = new HashMap<String, ArrayList<CombineFileSplit>>();
List<CombineFileSplit> retLists = new ArrayList<>();
Map<String, ArrayList<CombineFileSplit>> aliasToSplitList = new HashMap<>();
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
Map<Path, ArrayList<String>> pathToAliasesNoScheme = removeScheme(pathToAliases);
@@ -651,7 +644,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// 1. it serves more than one alias
// 2. the alias it serves is not sampled
// 3. it serves different alias than another path for the same split
if (l.size() != 1 || !nameToSamples.containsKey(l.get(0)) || (alias != null && l.get(0) != alias)) {
if (l.size() != 1 || !nameToSamples.containsKey(l.get(0)) || (alias != null && !Objects.equals(l.get(0), alias))) {
alias = null;
break;
}
@@ -662,7 +655,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
// split exclusively serves alias, which needs to be sampled
// add it to the split list of the alias.
if (!aliasToSplitList.containsKey(alias)) {
aliasToSplitList.put(alias, new ArrayList<CombineFileSplit>());
aliasToSplitList.put(alias, new ArrayList<>());
}
aliasToSplitList.get(alias).add(split);
} else {
@@ -727,7 +720,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
CombineHiveInputSplit hsplit = (CombineHiveInputSplit) split;
String inputFormatClassName = null;
Class inputFormatClass = null;
Class inputFormatClass;
try {
inputFormatClassName = hsplit.inputFormatClassName();
inputFormatClass = job.getClassByName(inputFormatClassName);
@@ -743,7 +736,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
static class CombineFilter implements PathFilter {
private final Set<String> pStrings = new HashSet<String>();
private final Set<String> pStrings = new HashSet<>();
// store a path prefix in this TestFilter
// PRECONDITION: p should always be a directory
@@ -764,7 +757,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
@Override
public boolean accept(Path path) {
boolean find = false;
while (path != null && !find) {
while (path != null) {
if (pStrings.contains(path.toUri().getPath())) {
find = true;
break;
@@ -838,19 +831,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
input = new HoodieParquetInputFormat();
}
input.setConf(job.getConfiguration());
result = new ArrayList<FileStatus>(Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
result = new ArrayList<>(Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
} else {
result = super.listStatus(job);
}
Iterator it = result.iterator();
while (it.hasNext()) {
FileStatus stat = (FileStatus) it.next();
if (!stat.isFile()) {
it.remove();
}
}
result.removeIf(stat -> !stat.isFile());
return result;
}
@@ -870,12 +856,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
InputSplit[] splits = super.getSplits(job, numSplits);
ArrayList inputSplitShims = new ArrayList();
List<InputSplitShim> inputSplitShims = new ArrayList<>();
for (int pos = 0; pos < splits.length; ++pos) {
CombineFileSplit split = (CombineFileSplit) splits[pos];
for (InputSplit inputSplit : splits) {
CombineFileSplit split = (CombineFileSplit) inputSplit;
if (split.getPaths().length > 0) {
inputSplitShims.add(new HadoopShimsSecure.InputSplitShim(job, split.getPaths(), split.getStartOffsets(),
inputSplitShims.add(new InputSplitShim(job, split.getPaths(), split.getStartOffsets(),
split.getLengths(), split.getLocations()));
}
}
@@ -884,7 +870,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
@Override
public HadoopShimsSecure.InputSplitShim getInputSplitShim() throws IOException {
public HadoopShimsSecure.InputSplitShim getInputSplitShim() {
return new HadoopShimsSecure.InputSplitShim();
}

View File

@@ -78,7 +78,7 @@ public abstract class AbstractRealtimeRecordReader {
// Property to set the max memory for dfs inputstream buffer size
public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size";
// Setting this to lower value of 1 MB since no control over how many RecordReaders will be started in a mapper
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1 * 1024 * 1024; // 1 MB
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1024 * 1024; // 1 MB
// Property to set file path prefix for spillable file
public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path";
// Default file path prefix for spillable file
@@ -170,18 +170,12 @@ public abstract class AbstractRealtimeRecordReader {
// /org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java#L188}
// Field Names -> {@link https://github.com/apache/hive/blob/f37c5de6c32b9395d1b34fa3c02ed06d1bfbf6eb/serde/src/java
// /org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java#L229}
Set<String> fieldOrdersSet = new LinkedHashSet<>();
String[] fieldOrdersWithDups = fieldOrderCsv.split(",");
for (String fieldOrder : fieldOrdersWithDups) {
fieldOrdersSet.add(fieldOrder);
}
String[] fieldOrders = fieldOrdersSet.toArray(new String[fieldOrdersSet.size()]);
Set<String> fieldOrdersSet = new LinkedHashSet<>(Arrays.asList(fieldOrdersWithDups));
String[] fieldOrders = fieldOrdersSet.toArray(new String[0]);
List<String> fieldNames = Arrays.stream(fieldNameCsv.split(","))
.filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList());
Set<String> fieldNamesSet = new LinkedHashSet<>();
for (String fieldName : fieldNames) {
fieldNamesSet.add(fieldName);
}
Set<String> fieldNamesSet = new LinkedHashSet<>(fieldNames);
// Hive does not provide ids for partitioning fields, so check for lengths excluding that.
if (fieldNamesSet.size() != fieldOrders.length) {
throw new HoodieException(String
@@ -189,7 +183,7 @@ public abstract class AbstractRealtimeRecordReader {
fieldNames.size(), fieldOrders.length));
}
TreeMap<Integer, String> orderedFieldMap = new TreeMap<>();
String[] fieldNamesArray = fieldNamesSet.toArray(new String[fieldNamesSet.size()]);
String[] fieldNamesArray = fieldNamesSet.toArray(new String[0]);
for (int ox = 0; ox < fieldOrders.length; ox++) {
orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNamesArray[ox]);
}
@@ -402,7 +396,7 @@ public abstract class AbstractRealtimeRecordReader {
public long getMaxCompactionMemoryInBytes() {
// jobConf.getMemoryForMapTask() returns in MB
return (long) Math
.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
.ceil(Double.parseDouble(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
* jobConf.getMemoryForMapTask() * 1024 * 1024L);
}
}

View File

@@ -108,7 +108,7 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
// for all unique split parents, obtain all delta files based on delta commit timeline,
// grouped on file id
List<HoodieRealtimeFileSplit> rtSplits = new ArrayList<>();
partitionsToParquetSplits.keySet().stream().forEach(partitionPath -> {
partitionsToParquetSplits.keySet().forEach(partitionPath -> {
// for each partition path obtain the data & log file groupings, then map back to inputsplits
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
@@ -149,7 +149,7 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
}
});
LOG.info("Returning a total splits of " + rtSplits.size());
return rtSplits.toArray(new InputSplit[rtSplits.size()]);
return rtSplits.toArray(new InputSplit[0]);
}
@Override

View File

@@ -48,7 +48,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, Ar
}
public static boolean canSkipMerging(JobConf jobConf) {
return Boolean.valueOf(jobConf.get(REALTIME_SKIP_MERGE_PROP, DEFAULT_REALTIME_SKIP_MERGE));
return Boolean.parseBoolean(jobConf.get(REALTIME_SKIP_MERGE_PROP, DEFAULT_REALTIME_SKIP_MERGE));
}
/**

View File

@@ -64,7 +64,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
split.getDeltaLogPaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(),
getMaxCompactionMemoryInBytes(),
Boolean
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
.parseBoolean(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH));
}

View File

@@ -79,7 +79,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
split.getBasePath(), split.getDeltaLogPaths(), getReaderSchema(), split.getMaxCommitTime(),
Boolean
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
.parseBoolean(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> {
// convert Hoodie log record to Hadoop AvroWritable and buffer
GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get();
@@ -93,7 +93,6 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
/**
* Setup log and parquet reading in parallel. Both write to central buffer.
*/
@SuppressWarnings("unchecked")
private List<BoundedInMemoryQueueProducer<ArrayWritable>> getParallelProducers() {
List<BoundedInMemoryQueueProducer<ArrayWritable>> producers = new ArrayList<>();
producers.add(new FunctionBasedQueueProducer<>(buffer -> {
@@ -105,7 +104,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
}
@Override
public boolean next(NullWritable key, ArrayWritable value) throws IOException {
public boolean next(NullWritable key, ArrayWritable value) {
if (!iterator.hasNext()) {
return false;
}
@@ -125,7 +124,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
}
@Override
public long getPos() throws IOException {
public long getPos() {
// TODO: vb - No logical way to represent parallel stream pos in a single long.
// Should we just return invalid (-1). Where is it used ?
return 0;

View File

@@ -33,12 +33,12 @@ import org.apache.parquet.avro.AvroParquetWriter;
import org.junit.rules.TemporaryFolder;
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.Objects;
import java.util.UUID;
public class InputFormatTestUtil {
@@ -59,13 +59,10 @@ public class InputFormatTestUtil {
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) {
List<File> dataFiles = Arrays.asList(Objects.requireNonNull(directory.listFiles((dir, name) -> {
String commitTs = FSUtils.getCommitTime(name);
return originalCommit.equals(commitTs);
}
}));
})));
if (randomize) {
Collections.shuffle(dataFiles);
}
@@ -183,16 +180,10 @@ public class InputFormatTestUtil {
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];
File fileToUpdate = Objects.requireNonNull(directory.listFiles((dir, name) -> 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 {
try (AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema)) {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, fileId)) {
if (numberOfRecordsToUpdate > 0) {
// update this record
@@ -203,8 +194,6 @@ public class InputFormatTestUtil {
}
parquetWriter.write(record);
}
} finally {
parquetWriter.close();
}
}

View File

@@ -157,7 +157,7 @@ public class InputPathHandlerTest {
@Test
public void testInputPathHandler() throws IOException {
inputPathHandler = new InputPathHandler(dfs.getConf(), inputPaths.toArray(
new Path[inputPaths.size()]), incrementalTables);
new Path[0]), incrementalTables);
List<Path> actualPaths = inputPathHandler.getGroupedIncrementalPaths().values().stream()
.flatMap(List::stream).collect(Collectors.toList());
assertTrue(actualComparesToExpected(actualPaths, incrementalPaths));

View File

@@ -122,7 +122,7 @@ public class TestHoodieParquetInputFormat {
throws IOException {
List<HoodieWriteStat> writeStats = HoodieTestUtils.generateFakeHoodieWriteStat(1);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
writeStats.stream().forEach(stat -> commitMetadata.addWriteStat(partitionPath, stat));
writeStats.forEach(stat -> commitMetadata.addWriteStat(partitionPath, stat));
File file = new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".commit");
file.createNewFile();
FileOutputStream fileOutputStream = new FileOutputStream(file);
@@ -221,8 +221,8 @@ public class TestHoodieParquetInputFormat {
String defaultmode = String.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, "db3.first_trips");
conf.set(defaultmode, HoodieHiveUtil.DEFAULT_SCAN_MODE);
List<String> actualincrTables = HoodieHiveUtil.getIncrementalTableNames(Job.getInstance(conf));
for (int i = 0; i < expectedincrTables.length; i++) {
assertTrue(actualincrTables.contains(expectedincrTables[i]));
for (String expectedincrTable : expectedincrTables) {
assertTrue(actualincrTables.contains(expectedincrTable));
}
}

View File

@@ -68,7 +68,7 @@ import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -167,9 +167,9 @@ public class TestHoodieRealtimeRecordReader {
private void setHiveColumnNameProps(List<Schema.Field> fields, JobConf jobConf, boolean isPartitioned) {
String names = fields.stream().map(Field::name).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).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, postions);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
String hiveOrderedColumnNames = fields.stream().filter(field -> !field.name().equalsIgnoreCase(PARTITION_COLUMN))
.map(Field::name).collect(Collectors.joining(","));
@@ -286,7 +286,7 @@ public class TestHoodieRealtimeRecordReader {
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);
basePath.getRoot().getPath(), Collections.singletonList(logFilePath), newCommitTime);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
@@ -361,7 +361,7 @@ public class TestHoodieRealtimeRecordReader {
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);
basePath.getRoot().getPath(), Collections.singletonList(logFilePath), newCommitTime);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(

View File

@@ -80,7 +80,7 @@ public class HiveSyncTool {
}
}
public void syncHoodieTable() throws ClassNotFoundException {
public void syncHoodieTable() {
try {
switch (hoodieHiveClient.getTableType()) {
case COPY_ON_WRITE:
@@ -193,7 +193,7 @@ public class HiveSyncTool {
.collect(Collectors.toList());
}
public static void main(String[] args) throws Exception {
public static void main(String[] args) {
// parse the params
final HiveSyncConfig cfg = new HiveSyncConfig();
JCommander cmd = new JCommander(cfg, null, args);

View File

@@ -18,7 +18,6 @@
package org.apache.hudi.hive;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -41,6 +40,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.metastore.api.Table;
@@ -65,7 +65,6 @@ import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -190,7 +189,7 @@ public class HoodieHiveClient {
for (int i = 0; i < syncConfig.partitionFields.size(); i++) {
partBuilder.add("`" + syncConfig.partitionFields.get(i) + "`='" + partitionValues.get(i) + "'");
}
return partBuilder.stream().collect(Collectors.joining(","));
return String.join(",", partBuilder);
}
private List<String> constructChangePartitions(String tableName, List<String> partitions) {
@@ -500,7 +499,7 @@ public class HoodieHiveClient {
* @param sql SQL statement to execute
*/
public CommandProcessorResponse updateHiveSQLUsingHiveDriver(String sql) throws HoodieHiveSyncException {
List<CommandProcessorResponse> responses = updateHiveSQLs(Arrays.asList(sql));
List<CommandProcessorResponse> responses = updateHiveSQLs(Collections.singletonList(sql));
return responses.get(responses.size() - 1);
}

View File

@@ -87,7 +87,7 @@ public abstract class ITTestBase {
cmd.add("hive.stats.autogather=false");
cmd.add("-e");
cmd.add("\"" + fullCommand + "\"");
return cmd.stream().toArray(String[]::new);
return cmd.toArray(new String[0]);
}
private static String getHiveConsoleCommandFile(String commandFile, String additionalVar) {

View File

@@ -43,7 +43,7 @@ import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -127,7 +127,7 @@ public class DataSourceUtils {
}
public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
checkPropNames.stream().forEach(prop -> {
checkPropNames.forEach(prop -> {
if (!props.containsKey(prop)) {
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
}
@@ -182,19 +182,13 @@ public class DataSourceUtils {
@SuppressWarnings("unchecked")
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
HoodieWriteConfig writeConfig, Option<EmbeddedTimelineService> timelineService) {
HoodieReadClient client = null;
try {
client = new HoodieReadClient<>(jssc, writeConfig, timelineService);
try (HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig, timelineService)) {
return client.tagLocation(incomingHoodieRecords)
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
} catch (TableNotFoundException e) {
// this will be executed when there is no hoodie table yet
// so no dups to drop
return incomingHoodieRecords;
} finally {
if (null != client) {
client.close();
}
}
}
@@ -207,12 +201,12 @@ public class DataSourceUtils {
}
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath) {
checkRequiredProperties(props, Arrays.asList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.basePath = basePath;
hiveSyncConfig.usePreApacheInputFormat =
props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(),
Boolean.valueOf(DataSourceWriteOptions.DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL()));
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL()));
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL());
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());

View File

@@ -52,7 +52,7 @@ object AvroConversionUtils {
}
def createRddForDeletes(df: DataFrame, rowField: String, partitionField: String): RDD[HoodieKey] = {
df.rdd.map(row => (new HoodieKey(row.getAs[String](rowField), row.getAs[String](partitionField))))
df.rdd.map(row => new HoodieKey(row.getAs[String](rowField), row.getAs[String](partitionField)))
}
def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss: SparkSession): Dataset[Row] = {
@@ -67,7 +67,7 @@ object AvroConversionUtils {
val convertor = AvroConversionHelper.createConverterToRow(schema, dataType)
records.map { x => convertor(x).asInstanceOf[Row] }
}
}, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr))).asInstanceOf[Dataset[Row]]
}, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr)))
}
}

View File

@@ -63,7 +63,7 @@ class DefaultSource extends RelationProvider
sqlContext.sparkContext.hadoopConfiguration.setClass(
"mapreduce.input.pathFilter.class",
classOf[HoodieROTablePathFilter],
classOf[org.apache.hadoop.fs.PathFilter]);
classOf[org.apache.hadoop.fs.PathFilter])
log.info("Constructing hoodie (as parquet) data source with options :" + parameters)
log.warn("Snapshot view not supported yet via data source, for MERGE_ON_READ tables. " +

View File

@@ -77,7 +77,7 @@ private[hudi] object HoodieSparkSqlWriter {
val jsc = new JavaSparkContext(sparkContext)
val basePath = new Path(parameters("path"))
val commitTime = HoodieActiveTimeline.createNewInstantTime();
val commitTime = HoodieActiveTimeline.createNewInstantTime()
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
var exists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
@@ -282,7 +282,7 @@ private[hudi] object HoodieSparkSqlWriter {
client.close()
commitSuccess && syncHiveSucess
} else {
log.error(s"$operation failed with ${errorCount} errors :");
log.error(s"$operation failed with $errorCount errors :")
if (log.isTraceEnabled) {
log.trace("Printing out the top 100 errors")
writeStatuses.rdd.filter(ws => ws.hasErrors)

View File

@@ -31,7 +31,7 @@ import org.scalatest.junit.AssertionsForJUnit
class TestDataSourceDefaults extends AssertionsForJUnit {
val schema = SchemaTestUtil.getComplexEvolvedSchema
var baseRecord: GenericRecord = null
var baseRecord: GenericRecord = _
@Before def initialize(): Unit = {
baseRecord = SchemaTestUtil
@@ -60,10 +60,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
new SimpleKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException => {
case e: IllegalArgumentException =>
// do nothing
}
};
// recordkey field not specified
try {
@@ -72,10 +71,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
new SimpleKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException => {
case e: IllegalArgumentException =>
// do nothing
}
};
// nested field as record key and partition path
val hk2 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin", "false"))
@@ -89,14 +87,13 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
.getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: HoodieException => {
case e: HoodieException =>
// do nothing
}
};
// if partition path can't be found, return default partition path
val hk3 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
.getKey(baseRecord);
.getKey(baseRecord)
assertEquals("default", hk3.getPartitionPath)
// if enable hive style partitioning
@@ -155,10 +152,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
new ComplexKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException => {
case e: IllegalArgumentException =>
// do nothing
}
};
// recordkey field not specified
try {
@@ -167,10 +163,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
new ComplexKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException => {
case e: IllegalArgumentException =>
// do nothing
}
};
// nested field as record key and partition path
val hk2 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
@@ -184,14 +179,13 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
.getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: HoodieException => {
case e: HoodieException =>
// do nothing
}
};
// if partition path can't be found, return default partition path
val hk3 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
.getKey(baseRecord);
.getKey(baseRecord)
assertEquals("default", hk3.getPartitionPath)
// if enable hive style partitioning
@@ -269,10 +263,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException => {
case e: IllegalArgumentException =>
// do nothing
}
};
// Nested record key not found
try {
@@ -280,10 +273,9 @@ class TestDataSourceDefaults extends AssertionsForJUnit {
.getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: HoodieException => {
case e: HoodieException =>
// do nothing
}
};
// if all parts of the composite record key are null/empty, throw error
try {

View File

@@ -350,8 +350,7 @@ public class FileSystemViewHandler {
} finally {
long endTs = System.currentTimeMillis();
long timeTakenMillis = endTs - beginTs;
LOG
.info(String.format(
LOG.info(String.format(
"TimeTakenMillis[Total=%d, Refresh=%d, handle=%d, Check=%d], "
+ "Success=%s, Query=%s, Host=%s, synced=%s",
timeTakenMillis, refreshCheckTimeTaken, handleTimeTaken, finalCheckTimeTaken, success,

View File

@@ -56,6 +56,7 @@ import java.time.ZoneId;
import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
@@ -81,7 +82,7 @@ public class HDFSParquetImporter implements Serializable {
this.cfg = cfg;
}
public static void main(String[] args) throws Exception {
public static void main(String[] args) {
final Config cfg = new Config();
JCommander cmd = new JCommander(cfg, null, args);
if (cfg.help || args.length == 0) {
@@ -160,8 +161,7 @@ public class HDFSParquetImporter implements Serializable {
AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr)));
ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class));
return jsc
.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, GenericRecord.class,
return jsc.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, GenericRecord.class,
job.getConfiguration())
// To reduce large number of tasks.
.coalesce(16 * cfg.parallelism).map(entry -> {
@@ -198,7 +198,7 @@ public class HDFSParquetImporter implements Serializable {
* @param <T> Type
*/
protected <T extends HoodieRecordPayload> JavaRDD<WriteStatus> load(HoodieWriteClient client, String instantTime,
JavaRDD<HoodieRecord<T>> hoodieRecords) throws Exception {
JavaRDD<HoodieRecord<T>> hoodieRecords) {
switch (cfg.command.toLowerCase()) {
case "upsert": {
return client.upsert(hoodieRecords, instantTime);
@@ -227,7 +227,7 @@ public class HDFSParquetImporter implements Serializable {
public static class FormatValidator implements IValueValidator<String> {
List<String> validFormats = Arrays.asList("parquet");
List<String> validFormats = Collections.singletonList("parquet");
@Override
public void validate(String name, String value) throws ParameterException {
@@ -241,7 +241,7 @@ public class HDFSParquetImporter implements Serializable {
public static class Config implements Serializable {
@Parameter(names = {"--command", "-c"}, description = "Write command Valid values are insert(default)/upsert/bulkinsert",
required = false, validateValueWith = CommandValidator.class)
validateValueWith = CommandValidator.class)
public String command = "INSERT";
@Parameter(names = {"--src-path", "-sp"}, description = "Base path for the input table", required = true)
public String srcPath = null;

View File

@@ -62,7 +62,6 @@ import java.util.stream.Collectors;
public class HiveIncrementalPuller {
private static final Logger LOG = LogManager.getLogger(HiveIncrementalPuller.class);
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
public static class Config implements Serializable {
@@ -97,6 +96,7 @@ public class HiveIncrementalPuller {
}
static {
String driverName = "org.apache.hive.jdbc.HiveDriver";
try {
Class.forName(driverName);
} catch (ClassNotFoundException e) {
@@ -219,8 +219,7 @@ public class HiveIncrementalPuller {
// Set the from commit time
executeStatement("set hoodie." + config.sourceTable + ".consume.start.timestamp=" + config.fromCommitTime, stmt);
// Set number of commits to pull
executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String.valueOf(config.maxCommits),
stmt);
executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + config.maxCommits, stmt);
}
private boolean deleteHDFSPath(FileSystem fs, String path) throws IOException {
@@ -233,14 +232,14 @@ public class HiveIncrementalPuller {
stmt.execute(sql);
}
private String inferCommitTime(FileSystem fs) throws SQLException, IOException {
private String inferCommitTime(FileSystem fs) throws IOException {
LOG.info("FromCommitTime not specified. Trying to infer it from Hoodie table " + config.targetDb + "."
+ config.targetTable);
String targetDataLocation = getTableLocation(config.targetDb, config.targetTable);
return scanForCommitTime(fs, targetDataLocation);
}
private String getTableLocation(String db, String table) throws SQLException {
private String getTableLocation(String db, String table) {
ResultSet resultSet = null;
Statement stmt = null;
try {
@@ -309,7 +308,7 @@ public class HiveIncrementalPuller {
return FileSystem.mkdirs(fs, targetBaseDirPath, new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
}
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation);
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants().map(HoodieInstant::getTimestamp)

View File

@@ -31,7 +31,6 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
@@ -45,11 +44,6 @@ public class HoodieCleaner {
*/
private final Config cfg;
/**
* Filesystem used.
*/
private transient FileSystem fs;
/**
* Spark context.
*/
@@ -60,22 +54,25 @@ public class HoodieCleaner {
*/
private TypedProperties props;
public HoodieCleaner(Config cfg, JavaSparkContext jssc) throws IOException {
public HoodieCleaner(Config cfg, JavaSparkContext jssc) {
this.cfg = cfg;
this.jssc = jssc;
this.fs = FSUtils.getFs(cfg.basePath, jssc.hadoopConfiguration());
/*
* Filesystem used.
*/
FileSystem fs = FSUtils.getFs(cfg.basePath, jssc.hadoopConfiguration());
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs)
: UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
LOG.info("Creating Cleaner with configs : " + props.toString());
}
public void run() throws Exception {
public void run() {
HoodieWriteConfig hoodieCfg = getHoodieClientConfig();
HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, false);
client.clean();
}
private HoodieWriteConfig getHoodieClientConfig() throws Exception {
private HoodieWriteConfig getHoodieClientConfig() {
return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.basePath).withAutoCommit(false)
.withProps(props).build();
}
@@ -101,7 +98,7 @@ public class HoodieCleaner {
public Boolean help = false;
}
public static void main(String[] args) throws Exception {
public static void main(String[] args) {
final Config cfg = new Config();
JCommander cmd = new JCommander(cfg, null, args);
if (cfg.help || args.length == 0) {

View File

@@ -60,8 +60,7 @@ public class HoodieCompactionAdminTool {
*/
public void run(JavaSparkContext jsc) throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath);
final CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath);
try {
try (CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath)) {
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {
throw new IllegalStateException("Output File Path already exists");
@@ -101,8 +100,6 @@ public class HoodieCompactionAdminTool {
default:
throw new IllegalStateException("Not yet implemented !!");
}
} finally {
admin.close();
}
}

View File

@@ -83,7 +83,7 @@ public class HoodieCompactor {
public List<String> configs = new ArrayList<>();
}
public static void main(String[] args) throws Exception {
public static void main(String[] args) {
final Config cfg = new Config();
JCommander cmd = new JCommander(cfg, null, args);
if (cfg.help || args.length == 0) {

View File

@@ -42,30 +42,28 @@ public class HoodieWithTimelineServer implements Serializable {
private final Config cfg;
private transient Javalin app = null;
public HoodieWithTimelineServer(Config cfg) {
this.cfg = cfg;
}
public static class Config implements Serializable {
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master")
public String sparkMaster = null;
@Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true)
public String sparkMemory = null;
@Parameter(names = {"--num-partitions", "-n"}, description = "Num Partitions", required = false)
@Parameter(names = {"--num-partitions", "-n"}, description = "Num Partitions")
public Integer numPartitions = 100;
@Parameter(names = {"--server-port", "-p"}, description = " Server Port", required = false)
@Parameter(names = {"--server-port", "-p"}, description = " Server Port")
public Integer serverPort = 26754;
@Parameter(names = {"--delay-secs", "-d"}, description = "Delay(sec) before client connects", required = false)
@Parameter(names = {"--delay-secs", "-d"}, description = "Delay(sec) before client connects")
public Integer delaySecs = 30;
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
}
public void startService() {
app = Javalin.create().start(cfg.serverPort);
Javalin app = Javalin.create().start(cfg.serverPort);
app.get("/", ctx -> ctx.result("Hello World"));
}
@@ -107,7 +105,7 @@ public class HoodieWithTimelineServer implements Serializable {
System.out.println("Response Code from(" + url + ") : " + response.getStatusLine().getStatusCode());
try (BufferedReader rd = new BufferedReader(new InputStreamReader(response.getEntity().getContent()))) {
StringBuffer result = new StringBuffer();
StringBuilder result = new StringBuilder();
String line;
while ((line = rd.readLine()) != null) {
result.append(line);

View File

@@ -167,9 +167,8 @@ public class UtilHelpers {
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec");
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
additionalConfigs.entrySet().forEach(e -> sparkConf.set(e.getKey(), e.getValue()));
SparkConf newSparkConf = HoodieWriteClient.registerClasses(sparkConf);
return newSparkConf;
additionalConfigs.forEach(sparkConf::set);
return HoodieWriteClient.registerClasses(sparkConf);
}
public static JavaSparkContext buildSparkContext(String appName, String defaultMaster, Map<String, String> configs) {
@@ -200,7 +199,7 @@ public class UtilHelpers {
* @param parallelism Parallelism
*/
public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) throws Exception {
int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) {
HoodieCompactionConfig compactionConfig = compactionStrategyClass
.map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false)
.withCompactionStrategy(ReflectionUtils.loadClass(strategy)).build())

View File

@@ -50,7 +50,7 @@ public class Compactor implements Serializable {
public void compact(HoodieInstant instant) throws IOException {
LOG.info("Compactor executing compaction " + instant);
JavaRDD<WriteStatus> res = compactionClient.compact(instant.getTimestamp());
long numWriteErrors = res.collect().stream().filter(r -> r.hasErrors()).count();
long numWriteErrors = res.collect().stream().filter(WriteStatus::hasErrors).count();
if (numWriteErrors != 0) {
// We treat even a single error in compaction as fatal
LOG.error("Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors);

View File

@@ -376,8 +376,8 @@ public class DeltaSync implements Serializable {
throw new HoodieDeltaStreamerException("Unknown operation :" + cfg.operation);
}
long totalErrorRecords = writeStatusRDD.mapToDouble(ws -> ws.getTotalErrorRecords()).sum().longValue();
long totalRecords = writeStatusRDD.mapToDouble(ws -> ws.getTotalRecords()).sum().longValue();
long totalErrorRecords = writeStatusRDD.mapToDouble(WriteStatus::getTotalErrorRecords).sum().longValue();
long totalRecords = writeStatusRDD.mapToDouble(WriteStatus::getTotalRecords).sum().longValue();
boolean hasErrors = totalErrorRecords > 0;
long hiveSyncTimeMs = 0;
if (!hasErrors || cfg.commitOnErrors) {
@@ -414,10 +414,10 @@ public class DeltaSync implements Serializable {
} else {
LOG.error("Delta Sync found errors when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords);
LOG.error("Printing out the top 100 errors");
writeStatusRDD.filter(ws -> ws.hasErrors()).take(100).forEach(ws -> {
writeStatusRDD.filter(WriteStatus::hasErrors).take(100).forEach(ws -> {
LOG.error("Global error :", ws.getGlobalError());
if (ws.getErrors().size() > 0) {
ws.getErrors().entrySet().forEach(r -> LOG.trace("Error for key:" + r.getKey() + " is " + r.getValue()));
ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " is " + value));
}
});
// Rolling back instant
@@ -456,7 +456,7 @@ public class DeltaSync implements Serializable {
/**
* Sync to Hive.
*/
private void syncHive() throws ClassNotFoundException {
private void syncHive() {
if (cfg.enableHiveSync) {
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath);
LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"

View File

@@ -64,7 +64,6 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
@@ -440,7 +439,7 @@ public class HoodieDeltaStreamer implements Serializable {
HoodieTableMetaClient meta =
new HoodieTableMetaClient(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);
List<HoodieInstant> pending = CompactionUtils.getPendingCompactionInstantTimes(meta);
pending.stream().forEach(hoodieInstant -> asyncCompactService.enqueuePendingCompaction(hoodieInstant));
pending.forEach(hoodieInstant -> asyncCompactService.enqueuePendingCompaction(hoodieInstant));
asyncCompactService.start((error) -> {
// Shutdown DeltaSync
shutdown(false);
@@ -554,8 +553,7 @@ public class HoodieDeltaStreamer implements Serializable {
@Override
protected Pair<CompletableFuture, ExecutorService> startService() {
ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentCompaction);
List<CompletableFuture<Boolean>> compactionFutures =
IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
try {
// Set Compactor Pool Name for allowing users to prioritize compaction
LOG.info("Setting Spark Pool name for compaction to " + SchedulerConfGenerator.COMPACT_POOL_NAME);
@@ -575,8 +573,7 @@ public class HoodieDeltaStreamer implements Serializable {
throw new HoodieIOException(e.getMessage(), e);
}
return true;
}, executor)).collect(Collectors.toList());
return Pair.of(CompletableFuture.allOf(compactionFutures.stream().toArray(CompletableFuture[]::new)), executor);
}, executor)).toArray(CompletableFuture[]::new)), executor);
}
}

View File

@@ -25,8 +25,8 @@ import com.codahale.metrics.Timer;
public class HoodieDeltaStreamerMetrics {
private HoodieWriteConfig config = null;
private String tableName = null;
private HoodieWriteConfig config;
private String tableName;
public String overallTimerName = null;
public String hiveSyncTimerName = null;

View File

@@ -46,7 +46,7 @@ public class SchedulerConfGenerator {
public static final String SPARK_SCHEDULER_MODE_KEY = "spark.scheduler.mode";
public static final String SPARK_SCHEDULER_ALLOCATION_FILE_KEY = "spark.scheduler.allocation.file";
private static String SPARK_SCHEDULING_PATTERN =
private static final String SPARK_SCHEDULING_PATTERN =
"<?xml version=\"1.0\"?>\n<allocations>\n <pool name=\"%s\">\n"
+ " <schedulingMode>%s</schedulingMode>\n <weight>%s</weight>\n <minShare>%s</minShare>\n"
+ " </pool>\n <pool name=\"%s\">\n <schedulingMode>%s</schedulingMode>\n"

View File

@@ -36,7 +36,7 @@ import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import java.util.Arrays;
import java.util.Collections;
public class HoodieIncrSource extends RowSource {
@@ -87,9 +87,9 @@ public class HoodieIncrSource extends RowSource {
@Override
public Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkptStr, long sourceLimit) {
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.HOODIE_SRC_BASE_PATH));
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.HOODIE_SRC_BASE_PATH));
/**
/*
* DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.HOODIE_SRC_BASE_PATH,
* Config.HOODIE_SRC_PARTITION_FIELDS)); List<String> partitionFields =
* props.getStringList(Config.HOODIE_SRC_PARTITION_FIELDS, ",", new ArrayList<>()); PartitionValueExtractor
@@ -121,7 +121,7 @@ public class HoodieIncrSource extends RowSource {
Dataset<Row> source = reader.load(srcPath);
/**
/*
* log.info("Partition Fields are : (" + partitionFields + "). Initial Source Schema :" + source.schema());
*
* StructType newSchema = new StructType(source.schema().fields()); for (String field : partitionFields) { newSchema

View File

@@ -25,7 +25,6 @@ import com.twitter.bijection.avro.GenericAvroCodecs;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import java.io.IOException;
import java.io.Serializable;
/**
@@ -80,7 +79,7 @@ public class AvroConvertor implements Serializable {
}
}
public GenericRecord fromJson(String json) throws IOException {
public GenericRecord fromJson(String json) {
initSchema();
initJsonConvertor();
return jsonConverter.convert(json, schema);

View File

@@ -33,13 +33,12 @@ import java.io.IOException;
import static org.junit.Assert.assertEquals;
public class TestTimestampBasedKeyGenerator {
private Schema schema;
private GenericRecord baseRecord;
private TypedProperties properties = new TypedProperties();
@Before
public void initialize() throws IOException {
schema = SchemaTestUtil.getTimestampEvolvedSchema();
Schema schema = SchemaTestUtil.getTimestampEvolvedSchema();
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1");