1
0

[checkstyle] Unify LOG form (#1092)

This commit is contained in:
lamber-ken
2019-12-10 19:23:38 +08:00
committed by leesf
parent 70a1040998
commit d447e2d751
100 changed files with 514 additions and 493 deletions

View File

@@ -43,7 +43,7 @@ import java.util.Map;
public class HoodieCommitMetadata implements Serializable {
public static final String SCHEMA_KEY = "schema";
private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class);
private static final Logger LOG = LogManager.getLogger(HoodieCommitMetadata.class);
protected Map<String, List<HoodieWriteStat>> partitionToWriteStats;
protected Boolean compacted;
@@ -118,7 +118,7 @@ public class HoodieCommitMetadata implements Serializable {
public String toJsonString() throws IOException {
if (partitionToWriteStats.containsKey(null)) {
log.info("partition path is null for " + partitionToWriteStats.get(null));
LOG.info("partition path is null for " + partitionToWriteStats.get(null));
partitionToWriteStats.remove(null);
}
return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this);

View File

@@ -51,7 +51,7 @@ public class HoodiePartitionMetadata {
private final FileSystem fs;
private static Logger log = LogManager.getLogger(HoodiePartitionMetadata.class);
private static final Logger LOG = LogManager.getLogger(HoodiePartitionMetadata.class);
/**
* Construct metadata from existing partition.
@@ -101,7 +101,7 @@ public class HoodiePartitionMetadata {
fs.rename(tmpMetaPath, metaPath);
}
} catch (IOException ioe) {
log.warn("Error trying to save partition metadata (this is okay, as long as " + "atleast 1 of these succced), "
LOG.warn("Error trying to save partition metadata (this is okay, as long as " + "atleast 1 of these succced), "
+ partitionPath, ioe);
} finally {
if (!metafileExists) {
@@ -111,7 +111,7 @@ public class HoodiePartitionMetadata {
fs.delete(tmpMetaPath, false);
}
} catch (IOException ioe) {
log.warn("Error trying to clean up temporary files for " + partitionPath, ioe);
LOG.warn("Error trying to clean up temporary files for " + partitionPath, ioe);
}
}
}

View File

@@ -31,7 +31,7 @@ import java.util.Map;
*/
public class HoodieRollingStatMetadata implements Serializable {
private static volatile Logger log = LogManager.getLogger(HoodieRollingStatMetadata.class);
private static final Logger LOG = LogManager.getLogger(HoodieRollingStatMetadata.class);
protected Map<String, Map<String, HoodieRollingStat>> partitionToRollingStats;
private String actionType = "DUMMY_ACTION";
public static final String ROLLING_STAT_METADATA_KEY = "ROLLING_STAT";
@@ -78,7 +78,7 @@ public class HoodieRollingStatMetadata implements Serializable {
public String toJsonString() throws IOException {
if (partitionToRollingStats.containsKey(null)) {
log.info("partition path is null for " + partitionToRollingStats.get(null));
LOG.info("partition path is null for " + partitionToRollingStats.get(null));
partitionToRollingStats.remove(null);
}
return HoodieCommitMetadata.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this);

View File

@@ -47,7 +47,7 @@ import java.util.stream.Collectors;
*/
public class HoodieTableConfig implements Serializable {
private static final transient Logger LOG = LogManager.getLogger(HoodieTableConfig.class);
private static final Logger LOG = LogManager.getLogger(HoodieTableConfig.class);
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";

View File

@@ -64,7 +64,7 @@ import java.util.stream.Collectors;
*/
public class HoodieTableMetaClient implements Serializable {
private static final transient Logger LOG = LogManager.getLogger(HoodieTableMetaClient.class);
private static final Logger LOG = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp";
public static String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux";

View File

@@ -67,7 +67,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
INFLIGHT_CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION,
REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
private static final transient Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
protected HoodieTableMetaClient metaClient;
private static AtomicReference<String> lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE));

View File

@@ -54,7 +54,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private HoodieTableMetaClient metaClient;
private Map<String, byte[]> readCommits = new HashMap<>();
private static final transient Logger LOG = LogManager.getLogger(HoodieArchivedTimeline.class);
private static final Logger LOG = LogManager.getLogger(HoodieArchivedTimeline.class);
public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) {
// Read back the commits to make sure

View File

@@ -47,7 +47,7 @@ import static java.util.Collections.reverse;
*/
public class HoodieDefaultTimeline implements HoodieTimeline {
private static final transient Logger LOG = LogManager.getLogger(HoodieDefaultTimeline.class);
private static final Logger LOG = LogManager.getLogger(HoodieDefaultTimeline.class);
private static final String HASHING_ALGORITHM = "SHA-256";

View File

@@ -66,7 +66,7 @@ import java.util.stream.Stream;
*/
public abstract class AbstractTableFileSystemView implements SyncableFileSystemView, Serializable {
private static Logger log = LogManager.getLogger(AbstractTableFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(AbstractTableFileSystemView.class);
protected HoodieTableMetaClient metaClient;
@@ -123,7 +123,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
});
long storePartitionsTs = timer.endTimer();
log.info("addFilesToView: NumFiles=" + statuses.length + ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
LOG.info("addFilesToView: NumFiles=" + statuses.length + ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
+ ", StoreTimeTaken=" + storePartitionsTs);
return fileGroups;
}
@@ -216,7 +216,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
if (!isPartitionAvailableInStore(partitionPathStr)) {
// Not loaded yet
try {
log.info("Building file system view for partition (" + partitionPathStr + ")");
LOG.info("Building file system view for partition (" + partitionPathStr + ")");
// Create the path if it does not exist already
Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr);
@@ -224,7 +224,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
long beginLsTs = System.currentTimeMillis();
FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
long endLsTs = System.currentTimeMillis();
log.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length + ", Time taken ="
LOG.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length + ", Time taken ="
+ (endLsTs - beginLsTs));
List<HoodieFileGroup> groups = addFilesToView(statuses);
@@ -235,10 +235,10 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
throw new HoodieIOException("Failed to list data files in partition " + partitionPathStr, e);
}
} else {
log.debug("View already built for Partition :" + partitionPathStr + ", FOUND is ");
LOG.debug("View already built for Partition :" + partitionPathStr + ", FOUND is ");
}
long endTs = System.currentTimeMillis();
log.info("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs));
LOG.info("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs));
return true;
});
}
@@ -289,7 +289,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
protected boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
log.info("Pending Compaction instant for (" + fileSlice + ") is :" + compactionWithInstantTime);
LOG.info("Pending Compaction instant for (" + fileSlice + ") is :" + compactionWithInstantTime);
return (compactionWithInstantTime.isPresent())
&& fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.get().getKey());
}
@@ -302,7 +302,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
protected FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) {
if (isFileSliceAfterPendingCompaction(fileSlice)) {
log.info("File Slice (" + fileSlice + ") is in pending compaction");
LOG.info("File Slice (" + fileSlice + ") is in pending compaction");
// Data file is filtered out of the file-slice as the corresponding compaction
// instant not completed yet.
FileSlice transformed =

View File

@@ -51,7 +51,7 @@ import java.util.concurrent.ConcurrentHashMap;
* clients for querying.
*/
public class FileSystemViewManager {
private static Logger logger = LogManager.getLogger(FileSystemViewManager.class);
private static final Logger LOG = LogManager.getLogger(FileSystemViewManager.class);
private final SerializableConfiguration conf;
// The View Storage config used to store file-system views
@@ -126,7 +126,7 @@ public class FileSystemViewManager {
*/
private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) {
logger.info("Creating SpillableMap based view for basePath " + basePath);
LOG.info("Creating SpillableMap based view for basePath " + basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf);
@@ -142,7 +142,7 @@ public class FileSystemViewManager {
*/
private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) {
logger.info("Creating InMemory based view for basePath " + basePath);
LOG.info("Creating InMemory based view for basePath " + basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled());
@@ -158,7 +158,7 @@ public class FileSystemViewManager {
*/
private static RemoteHoodieTableFileSystemView createRemoteFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
logger.info("Creating remote view for basePath " + metaClient.getBasePath() + ". Server="
LOG.info("Creating remote view for basePath " + metaClient.getBasePath() + ". Server="
+ viewConf.getRemoteViewServerHost() + ":" + viewConf.getRemoteViewServerPort());
return new RemoteHoodieTableFileSystemView(viewConf.getRemoteViewServerHost(), viewConf.getRemoteViewServerPort(),
metaClient);
@@ -173,26 +173,26 @@ public class FileSystemViewManager {
*/
public static FileSystemViewManager createViewManager(final SerializableConfiguration conf,
final FileSystemViewStorageConfig config) {
logger.info("Creating View Manager with storage type :" + config.getStorageType());
LOG.info("Creating View Manager with storage type :" + config.getStorageType());
switch (config.getStorageType()) {
case EMBEDDED_KV_STORE:
logger.info("Creating embedded rocks-db based Table View");
LOG.info("Creating embedded rocks-db based Table View");
return new FileSystemViewManager(conf, config,
(basePath, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, basePath));
case SPILLABLE_DISK:
logger.info("Creating Spillable Disk based Table View");
LOG.info("Creating Spillable Disk based Table View");
return new FileSystemViewManager(conf, config,
(basePath, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, basePath));
case MEMORY:
logger.info("Creating in-memory based Table View");
LOG.info("Creating in-memory based Table View");
return new FileSystemViewManager(conf, config,
(basePath, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, basePath));
case REMOTE_ONLY:
logger.info("Creating remote only table view");
LOG.info("Creating remote only table view");
return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> createRemoteFileSystemView(conf,
viewConfig, new HoodieTableMetaClient(conf.newCopy(), basePath)));
case REMOTE_FIRST:
logger.info("Creating remote first table view");
LOG.info("Creating remote first table view");
return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> {
RemoteHoodieTableFileSystemView remoteFileSystemView =
createRemoteFileSystemView(conf, viewConfig, new HoodieTableMetaClient(conf.newCopy(), basePath));

View File

@@ -48,7 +48,7 @@ import java.util.stream.Stream;
*/
public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystemView {
private static Logger log = LogManager.getLogger(HoodieTableFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(HoodieTableFileSystemView.class);
// mapping from partition paths to file groups contained within them
protected Map<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
@@ -193,7 +193,7 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
@Override
protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
log.info("Adding file-groups for partition :" + partitionPath + ", #FileGroups=" + fileGroups.size());
LOG.info("Adding file-groups for partition :" + partitionPath + ", #FileGroups=" + fileGroups.size());
List<HoodieFileGroup> newList = new ArrayList<>(fileGroups);
partitionToFileGroupsMap.put(partitionPath, newList);
}

View File

@@ -54,7 +54,7 @@ import java.util.stream.Collectors;
*/
public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTableFileSystemView {
private static Logger log = LogManager.getLogger(IncrementalTimelineSyncFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(IncrementalTimelineSyncFileSystemView.class);
// Allows incremental Timeline syncing
private final boolean incrementalTimelineSyncEnabled;
@@ -78,19 +78,19 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
if (incrementalTimelineSyncEnabled) {
TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline);
if (diffResult.canSyncIncrementally()) {
log.info("Doing incremental sync");
LOG.info("Doing incremental sync");
runIncrementalSync(newTimeline, diffResult);
log.info("Finished incremental sync");
LOG.info("Finished incremental sync");
// Reset timeline to latest
refreshTimeline(newTimeline);
return;
}
}
} catch (Exception ioe) {
log.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe);
LOG.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe);
}
log.warn("Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing");
LOG.warn("Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing");
super.runSync(oldTimeline, newTimeline);
}
@@ -102,7 +102,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
*/
private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diffResult) {
log.info("Timeline Diff Result is :" + diffResult);
LOG.info("Timeline Diff Result is :" + diffResult);
// First remove pending compaction instants which were completed
diffResult.getFinishedCompactionInstants().stream().forEach(instant -> {
@@ -143,7 +143,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* @param instant Compaction Instant to be removed
*/
private void removePendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Removing completed compaction instant (" + instant + ")");
LOG.info("Removing completed compaction instant (" + instant + ")");
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
removePendingCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan)
.map(instantPair -> Pair.of(instantPair.getValue().getKey(),
@@ -157,7 +157,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* @param instant Compaction Instant
*/
private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing pending compaction instant (" + instant + ")");
LOG.info("Syncing pending compaction instant (" + instant + ")");
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
List<Pair<String, CompactionOperation>> pendingOps =
CompactionUtils.getPendingCompactionOperations(instant, compactionPlan)
@@ -188,13 +188,13 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* @param instant Instant
*/
private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing committed instant (" + instant + ")");
LOG.info("Syncing committed instant (" + instant + ")");
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
commitMetadata.getPartitionToWriteStats().entrySet().stream().forEach(entry -> {
String partition = entry.getKey();
if (isPartitionAvailableInStore(partition)) {
log.info("Syncing partition (" + partition + ") of instant (" + instant + ")");
LOG.info("Syncing partition (" + partition + ") of instant (" + instant + ")");
FileStatus[] statuses = entry.getValue().stream().map(p -> {
FileStatus status = new FileStatus(p.getFileSizeInBytes(), false, 0, 0, 0, 0, null, null, null,
new Path(String.format("%s/%s", metaClient.getBasePath(), p.getPath())));
@@ -204,10 +204,10 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
buildFileGroups(statuses, timeline.filterCompletedAndCompactionInstants(), false);
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.ADD);
} else {
log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
LOG.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
}
});
log.info("Done Syncing committed instant (" + instant + ")");
LOG.info("Done Syncing committed instant (" + instant + ")");
}
/**
@@ -217,7 +217,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* @param instant Restore Instant
*/
private void addRestoreInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing restore instant (" + instant + ")");
LOG.info("Syncing restore instant (" + instant + ")");
HoodieRestoreMetadata metadata =
AvroUtils.deserializeAvroMetadata(timeline.getInstantDetails(instant).get(), HoodieRestoreMetadata.class);
@@ -231,7 +231,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
removeFileSlicesForPartition(timeline, instant, e.getKey(),
e.getValue().stream().map(x -> x.getValue()).collect(Collectors.toList()));
});
log.info("Done Syncing restore instant (" + instant + ")");
LOG.info("Done Syncing restore instant (" + instant + ")");
}
/**
@@ -241,14 +241,14 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* @param instant Rollback Instant
*/
private void addRollbackInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing rollback instant (" + instant + ")");
LOG.info("Syncing rollback instant (" + instant + ")");
HoodieRollbackMetadata metadata =
AvroUtils.deserializeAvroMetadata(timeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
metadata.getPartitionMetadata().entrySet().stream().forEach(e -> {
removeFileSlicesForPartition(timeline, instant, e.getKey(), e.getValue().getSuccessDeleteFiles());
});
log.info("Done Syncing rollback instant (" + instant + ")");
LOG.info("Done Syncing rollback instant (" + instant + ")");
}
/**
@@ -258,7 +258,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* @param instant Clean instant
*/
private void addCleanInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
log.info("Syncing cleaner instant (" + instant + ")");
LOG.info("Syncing cleaner instant (" + instant + ")");
HoodieCleanMetadata cleanMetadata =
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get());
cleanMetadata.getPartitionMetadata().entrySet().stream().forEach(entry -> {
@@ -270,13 +270,13 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
.collect(Collectors.toList());
removeFileSlicesForPartition(timeline, instant, entry.getKey(), fullPathList);
});
log.info("Done Syncing cleaner instant (" + instant + ")");
LOG.info("Done Syncing cleaner instant (" + instant + ")");
}
private void removeFileSlicesForPartition(HoodieTimeline timeline, HoodieInstant instant, String partition,
List<String> paths) {
if (isPartitionAvailableInStore(partition)) {
log.info("Removing file slices for partition (" + partition + ") for instant (" + instant + ")");
LOG.info("Removing file slices for partition (" + partition + ") for instant (" + instant + ")");
FileStatus[] statuses = paths.stream().map(p -> {
FileStatus status = new FileStatus();
status.setPath(new Path(p));
@@ -286,7 +286,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
buildFileGroups(statuses, timeline.filterCompletedAndCompactionInstants(), false);
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.REMOVE);
} else {
log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
LOG.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
}
}
@@ -309,7 +309,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
DeltaApplyMode mode) {
if (deltaFileGroups.isEmpty()) {
log.info("No delta file groups for partition :" + partition);
LOG.info("No delta file groups for partition :" + partition);
return;
}

View File

@@ -45,7 +45,7 @@ import java.util.stream.Stream;
*/
public class PriorityBasedFileSystemView implements SyncableFileSystemView, Serializable {
private static Logger log = LogManager.getLogger(PriorityBasedFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(PriorityBasedFileSystemView.class);
private final SyncableFileSystemView preferredView;
private final SyncableFileSystemView secondaryView;
@@ -59,13 +59,13 @@ public class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri
private <R> R execute(Function0<R> preferredFunction, Function0<R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
LOG.warn("Routing request to secondary file-system view");
return secondaryFunction.apply();
} else {
try {
return preferredFunction.apply();
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
LOG.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply();
}
@@ -74,13 +74,13 @@ public class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri
private <T1, R> R execute(T1 val, Function1<T1, R> preferredFunction, Function1<T1, R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
LOG.warn("Routing request to secondary file-system view");
return secondaryFunction.apply(val);
} else {
try {
return preferredFunction.apply(val);
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
LOG.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply(val);
}
@@ -90,13 +90,13 @@ public class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri
private <T1, T2, R> R execute(T1 val, T2 val2, Function2<T1, T2, R> preferredFunction,
Function2<T1, T2, R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
LOG.warn("Routing request to secondary file-system view");
return secondaryFunction.apply(val, val2);
} else {
try {
return preferredFunction.apply(val, val2);
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
LOG.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply(val, val2);
}
@@ -106,13 +106,13 @@ public class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri
private <T1, T2, T3, R> R execute(T1 val, T2 val2, T3 val3, Function3<T1, T2, T3, R> preferredFunction,
Function3<T1, T2, T3, R> secondaryFunction) {
if (errorOnPreferredView) {
log.warn("Routing request to secondary file-system view");
LOG.warn("Routing request to secondary file-system view");
return secondaryFunction.apply(val, val2, val3);
} else {
try {
return preferredFunction.apply(val, val2, val3);
} catch (RuntimeException re) {
log.error("Got error running preferred function. Trying secondary", re);
LOG.error("Got error running preferred function. Trying secondary", re);
errorOnPreferredView = true;
return secondaryFunction.apply(val, val2, val3);
}

View File

@@ -108,7 +108,7 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
public static final String INCLUDE_FILES_IN_PENDING_COMPACTION_PARAM = "includependingcompaction";
private static Logger log = LogManager.getLogger(RemoteHoodieTableFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(RemoteHoodieTableFileSystemView.class);
private final String serverHost;
private final int serverPort;
@@ -148,7 +148,7 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
builder.addParameter(TIMELINE_HASH, timeline.getTimelineHash());
String url = builder.toString();
log.info("Sending request : (" + url + ")");
LOG.info("Sending request : (" + url + ")");
Response response = null;
int timeout = 1000 * 300; // 5 min timeout
switch (method) {

View File

@@ -58,7 +58,7 @@ import java.util.stream.Stream;
*/
public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSystemView {
private static Logger log = LogManager.getLogger(RocksDbBasedFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(RocksDbBasedFileSystemView.class);
private final FileSystemViewStorageConfig config;
@@ -87,7 +87,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
schemaHelper.getAllColumnFamilies().stream().forEach(rocksDB::addColumnFamily);
super.init(metaClient, visibleActiveTimeline);
log.info("Created ROCKSDB based file-system view at " + config.getRocksdbBasePath());
LOG.info("Created ROCKSDB based file-system view at " + config.getRocksdbBasePath());
}
@Override
@@ -102,7 +102,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
schemaHelper.getKeyForPendingCompactionLookup(opPair.getValue().getFileGroupId()), opPair);
});
log.info("Initializing pending compaction operations. Count=" + batch.count());
LOG.info("Initializing pending compaction operations. Count=" + batch.count());
});
}
@@ -135,7 +135,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
protected void resetViewState() {
log.info("Deleting all rocksdb data associated with dataset filesystem view");
LOG.info("Deleting all rocksdb data associated with dataset filesystem view");
rocksDB.close();
rocksDB = new RocksDBDAO(metaClient.getBasePath(), config.getRocksdbBasePath());
}
@@ -157,7 +157,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
log.info("Resetting and adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
LOG.info("Resetting and adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
+ config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
@@ -184,7 +184,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
// record that partition is loaded.
rocksDB.put(schemaHelper.getColFamilyForStoredPartitions(), lookupKey, Boolean.TRUE);
log.info("Finished adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
LOG.info("Finished adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
+ config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
}
@@ -202,7 +202,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
return fs;
} else {
// First remove the file-slice
log.info("Removing old Slice in DB. FS=" + oldSlice);
LOG.info("Removing old Slice in DB. FS=" + oldSlice);
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
schemaHelper.getKeyForSliceView(fg, oldSlice));
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
@@ -224,11 +224,11 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
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);
LOG.info("Adding back new File Slice after add FS=" + newFileSlice);
return newFileSlice;
}
case REMOVE: {
log.info("Removing old File Slice =" + fs);
LOG.info("Removing old File Slice =" + fs);
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
fs.getDataFile().orElseGet(() -> {
oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
@@ -239,7 +239,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
// Add remaining log files back
logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
if (newFileSlice.getDataFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
log.info("Adding back new file-slice after remove FS=" + newFileSlice);
LOG.info("Adding back new file-slice after remove FS=" + newFileSlice);
return newFileSlice;
}
return null;
@@ -335,10 +335,10 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
public void close() {
log.info("Closing Rocksdb !!");
LOG.info("Closing Rocksdb !!");
closed = true;
rocksDB.close();
log.info("Closed Rocksdb !!");
LOG.info("Closed Rocksdb !!");
}
@Override

View File

@@ -42,7 +42,7 @@ import java.util.stream.Stream;
*/
public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
private static Logger log = LogManager.getLogger(SpillableMapBasedFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(SpillableMapBasedFileSystemView.class);
private final long maxMemoryForFileGroupMap;
private final long maxMemoryForPendingCompaction;
@@ -66,7 +66,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
@Override
protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
try {
log.info("Creating Partition To File groups map using external spillable Map. Max Mem=" + maxMemoryForFileGroupMap
LOG.info("Creating Partition To File groups map using external spillable Map. Max Mem=" + maxMemoryForFileGroupMap
+ ", BaseDir=" + baseStoreDir);
new File(baseStoreDir).mkdirs();
return (Map<String, List<HoodieFileGroup>>) (new ExternalSpillableMap<>(maxMemoryForFileGroupMap, baseStoreDir,
@@ -79,7 +79,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingCompactionMap(
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction) {
try {
log.info("Creating Pending Compaction map using external spillable Map. Max Mem=" + maxMemoryForPendingCompaction
LOG.info("Creating Pending Compaction map using external spillable Map. Max Mem=" + maxMemoryForPendingCompaction
+ ", BaseDir=" + baseStoreDir);
new File(baseStoreDir).mkdirs();
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> pendingMap = new ExternalSpillableMap<>(

View File

@@ -41,7 +41,7 @@ import java.util.Set;
*/
public class DFSPropertiesConfiguration {
private static volatile Logger log = LogManager.getLogger(DFSPropertiesConfiguration.class);
private static final Logger LOG = LogManager.getLogger(DFSPropertiesConfiguration.class);
private final FileSystem fs;
@@ -76,7 +76,7 @@ public class DFSPropertiesConfiguration {
BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file)));
addProperties(reader);
} catch (IOException ioe) {
log.error("Error reading in properies from dfs", ioe);
LOG.error("Error reading in properies from dfs", ioe);
throw new IllegalArgumentException("Cannot read properties from dfs", ioe);
}
}

View File

@@ -39,7 +39,7 @@ import java.util.stream.Collectors;
*/
public class FailSafeConsistencyGuard implements ConsistencyGuard {
private static final transient Logger LOG = LogManager.getLogger(FailSafeConsistencyGuard.class);
private static final Logger LOG = LogManager.getLogger(FailSafeConsistencyGuard.class);
private final FileSystem fs;
private final ConsistencyGuardConfig consistencyGuardConfig;

View File

@@ -33,7 +33,7 @@ import org.apache.log4j.Logger;
*/
public class HoodieRecordSizeEstimator<T extends HoodieRecordPayload> implements SizeEstimator<HoodieRecord<T>> {
private static Logger log = LogManager.getLogger(HoodieRecordSizeEstimator.class);
private static final Logger LOG = LogManager.getLogger(HoodieRecordSizeEstimator.class);
// Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa
private final Schema schema;
@@ -50,7 +50,7 @@ public class HoodieRecordSizeEstimator<T extends HoodieRecordPayload> implements
/** {@link ExternalSpillableMap} **/
long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord);
long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema);
log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema);
LOG.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema);
return sizeOfRecord;
}
}

View File

@@ -55,7 +55,7 @@ import java.util.stream.Stream;
*/
public class RocksDBDAO {
protected static final transient Logger LOG = LogManager.getLogger(RocksDBDAO.class);
private static final Logger LOG = LogManager.getLogger(RocksDBDAO.class);
private transient ConcurrentHashMap<String, ColumnFamilyHandle> managedHandlesMap;
private transient ConcurrentHashMap<String, ColumnFamilyDescriptor> managedDescriptorMap;

View File

@@ -36,7 +36,7 @@ import java.util.stream.Collectors;
*/
public class TimelineDiffHelper {
protected static Logger log = LogManager.getLogger(TimelineDiffHelper.class);
private static final Logger LOG = LogManager.getLogger(TimelineDiffHelper.class);
public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline oldTimeline,
HoodieTimeline newTimeline) {
@@ -64,7 +64,7 @@ public class TimelineDiffHelper {
if (!lostPendingCompactions.isEmpty()) {
// If a compaction is unscheduled, fall back to complete refresh of fs view since some log files could have been
// moved. Its unsafe to incrementally sync in that case.
log.warn("Some pending compactions are no longer in new timeline (unscheduled ?)." + "They are :"
LOG.warn("Some pending compactions are no longer in new timeline (unscheduled ?)." + "They are :"
+ lostPendingCompactions);
return TimelineDiffResult.UNSAFE_SYNC_RESULT;
}
@@ -77,7 +77,7 @@ public class TimelineDiffHelper {
return new TimelineDiffResult(newInstants, finishedCompactionInstants, true);
} else {
// One or more timelines is empty
log.warn("One or more timelines is empty");
LOG.warn("One or more timelines is empty");
return TimelineDiffResult.UNSAFE_SYNC_RESULT;
}
}

View File

@@ -44,7 +44,7 @@ import java.util.stream.Collectors;
*/
public class BoundedInMemoryExecutor<I, O, E> {
private static Logger logger = LogManager.getLogger(BoundedInMemoryExecutor.class);
private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class);
// Executor service used for launching writer thread.
private final ExecutorService executorService;
@@ -91,7 +91,7 @@ public class BoundedInMemoryExecutor<I, O, E> {
preExecute();
producer.produce(queue);
} catch (Exception e) {
logger.error("error producing records", e);
LOG.error("error producing records", e);
queue.markAsFailed(e);
throw e;
} finally {
@@ -115,14 +115,14 @@ public class BoundedInMemoryExecutor<I, O, E> {
private Future<E> startConsumer() {
return consumer.map(consumer -> {
return executorService.submit(() -> {
logger.info("starting consumer thread");
LOG.info("starting consumer thread");
preExecute();
try {
E result = consumer.consume(queue);
logger.info("Queue Consumption is done; notifying producer threads");
LOG.info("Queue Consumption is done; notifying producer threads");
return result;
} catch (Exception e) {
logger.error("error consuming records", e);
LOG.error("error consuming records", e);
queue.markAsFailed(e);
throw e;
}

View File

@@ -58,7 +58,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
public static final int RECORD_SAMPLING_RATE = 64;
// maximum records that will be cached
private static final int RECORD_CACHING_LIMIT = 128 * 1024;
private static Logger logger = LogManager.getLogger(BoundedInMemoryQueue.class);
private static final Logger LOG = LogManager.getLogger(BoundedInMemoryQueue.class);
// It indicates number of records to cache. We will be using sampled record's average size to
// determine how many
// records we should cache and will change (increase/decrease) permits accordingly.
@@ -203,7 +203,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
break;
}
} catch (InterruptedException e) {
logger.error("error reading records from queue", e);
LOG.error("error reading records from queue", e);
throw new HoodieException(e);
}
}

View File

@@ -71,7 +71,7 @@ import static org.junit.Assert.assertTrue;
@SuppressWarnings("ResultOfMethodCallIgnored")
public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
private static final transient Logger LOG = LogManager.getLogger(TestHoodieTableFileSystemView.class);
private static final Logger LOG = LogManager.getLogger(TestHoodieTableFileSystemView.class);
private static String TEST_WRITE_TOKEN = "1-0-1";

View File

@@ -77,7 +77,7 @@ import static org.apache.hudi.common.table.HoodieTimeline.COMPACTION_ACTION;
*/
public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
private static final transient Logger LOG = LogManager.getLogger(TestIncrementalFSViewSync.class);
private static final Logger LOG = LogManager.getLogger(TestIncrementalFSViewSync.class);
private static String TEST_WRITE_TOKEN = "1-0-1";