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

@@ -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())
.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;
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();
}
/**
@@ -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,
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}));
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));
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)))
.longValue();
return reservedForPendingComaction;
return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION)))
.longValue();
}
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,39 +98,39 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
rocksDB.writeBatch(batch -> {
operations.forEach(opPair -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opPair.getValue().getFileGroupId()), opPair);
});
operations.forEach(opPair ->
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
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 -> {
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);
});
});
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 -> {
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(
getPendingCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null,
"Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
+ opInstantPair.getValue().getFileGroupId());
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()));
});
});
rocksDB.writeBatch(batch ->
operations.forEach(opInstantPair -> {
Preconditions.checkArgument(
getPendingCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null,
"Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
+ 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 -> {
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);
});
});
});
});
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)
);
})
)
);
// record that partition is loaded.
rocksDB.put(schemaHelper.getColFamilyForStoredPartitions(), lookupKey, Boolean.TRUE);
@@ -194,69 +193,66 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
*/
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
DeltaApplyMode mode) {
rocksDB.writeBatch(batch -> {
deltaFileGroups.stream().forEach(fg -> {
fg.getAllRawFileSlices().map(fs -> {
FileSlice oldSlice = getFileSlice(partition, fs.getFileId(), fs.getBaseInstantTime());
if (null == oldSlice) {
return fs;
} 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.writeBatch(batch ->
deltaFileGroups.forEach(fg ->
fg.getAllRawFileSlices().map(fs -> {
FileSlice oldSlice = getFileSlice(partition, fs.getFileId(), fs.getBaseInstantTime());
if (null == oldSlice) {
return fs;
} 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));
Map<String, HoodieLogFile> logFiles = oldSlice.getLogFiles()
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
Map<String, HoodieLogFile> deltaLogFiles =
fs.getLogFiles().map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
Map<String, HoodieLogFile> logFiles = oldSlice.getLogFiles()
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
Map<String, HoodieLogFile> deltaLogFiles =
fs.getLogFiles().map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
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));
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));
LOG.info("Adding back new File Slice after add FS=" + newFileSlice);
return newFileSlice;
}
case REMOVE: {
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));
return null;
});
switch (mode) {
case ADD: {
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
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().forEach(newFileSlice::addLogFile);
LOG.info("Adding back new File Slice after add FS=" + newFileSlice);
return newFileSlice;
}
case REMOVE: {
LOG.info("Removing old File Slice =" + fs);
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
fs.getBaseFile().orElseGet(() -> {
oldSlice.getBaseFile().ifPresent(newFileSlice::setBaseFile);
return null;
});
deltaLogFiles.keySet().stream().forEach(p -> logFiles.remove(p));
// Add remaining log files back
logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
if (newFileSlice.getBaseFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
LOG.info("Adding back new file-slice after remove FS=" + newFileSlice);
return newFileSlice;
deltaLogFiles.keySet().forEach(logFiles::remove);
// Add remaining log files back
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;
}
return null;
}
default:
throw new IllegalStateException("Unknown diff apply mode=" + mode);
}
return null;
}
default:
throw new IllegalStateException("Unknown diff apply mode=" + mode);
}
}
}).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);
});
});
});
});
}).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)
);
})
)
);
}
@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,25 +157,23 @@ public class CompactionUtils {
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToPendingCompactionWithInstantMap =
new HashMap<>();
pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> {
return 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();
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();
if (!operation.equals(anotherOperation)) {
String msg = "Hudi File Id (" + pair.getKey() + ") has more than 1 pending compactions. Instants: "
+ pair.getValue() + ", " + fgIdToPendingCompactionWithInstantMap.get(pair.getKey());
throw new IllegalStateException(msg);
}
}
fgIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue());
});
if (!operation.equals(anotherOperation)) {
String msg = "Hudi File Id (" + pair.getKey() + ") has more than 1 pending compactions. Instants: "
+ pair.getValue() + ", " + fgIdToPendingCompactionWithInstantMap.get(pair.getKey());
throw new IllegalStateException(msg);
}
}
fgIdToPendingCompactionWithInstantMap.put(pair.getKey(), pair.getValue());
});
return fgIdToPendingCompactionWithInstantMap;
}
@@ -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);
}