Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-8786] Running intellij code cleanup #12524

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -713,10 +713,11 @@ public void managePartitionIndexes(String tableName) throws ExecutionException,
boolean indexesChanges = false;
for (PartitionIndexDescriptor existingIdx: existingIdxsResp.partitionIndexDescriptorList()) {
List<String> idxColumns = existingIdx.keys().stream().map(key -> key.name()).collect(Collectors.toList());
Boolean toBeRemoved = true;
boolean toBeRemoved = true;
for (List<String> neededIdx : partitionsIndexNeeded) {
if (neededIdx.equals(idxColumns)) {
toBeRemoved = false;
break;
}
}
if (toBeRemoved) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,9 @@
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.InstantComparator;
import org.apache.hudi.common.table.timeline.TimelineUtils;
import org.apache.hudi.common.util.NumericUtils;
Expand Down Expand Up @@ -194,7 +194,7 @@ public String showArchivedCommits(
HoodieArchivedTimeline archivedTimeline = HoodieCLI.getTableMetaClient().getArchivedTimeline();
try {
archivedTimeline.loadInstantDetailsInMemory(startTs, endTs);
HoodieTimeline timelineRange = (HoodieTimeline)archivedTimeline.findInstantsInRange(startTs, endTs);
HoodieTimeline timelineRange = archivedTimeline.findInstantsInRange(startTs, endTs);
if (includeExtraMetadata) {
return printCommitsWithMetadata(timelineRange, limit, sortByField, descending, headerOnly, exportTableName, partition);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,10 @@
import org.apache.hudi.client.CompactionAdminClient.ValidationOpResult;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.InstantGenerator;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.Option;
Expand Down Expand Up @@ -431,7 +431,7 @@ protected static String printCompaction(HoodieCompactionPlan compactionPlan,
}

private static String getTmpSerializerFile() {
return TMP_DIR + UUID.randomUUID().toString() + ".ser";
return TMP_DIR + UUID.randomUUID() + ".ser";
}

private <T> T deSerializeOperationResult(StoragePath inputPath,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@

import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.BaseHoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.BaseHoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieTimeline;

import static org.apache.hudi.cli.utils.CommitUtil.getTimeDaysAgo;
import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
Expand Down Expand Up @@ -57,7 +57,7 @@ public static HoodieTimeline getTimelineInRange(String startTs, String endTs, bo
if (includeArchivedTimeline) {
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
archivedTimeline.loadInstantDetailsInMemory(startTs, endTs);
return ((HoodieTimeline)archivedTimeline.findInstantsInRange(startTs, endTs)).mergeTimeline(activeTimeline);
return archivedTimeline.findInstantsInRange(startTs, endTs).mergeTimeline(activeTimeline);
}
return activeTimeline;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
public class InputStreamConsumer extends Thread {

private static final Logger LOG = LoggerFactory.getLogger(InputStreamConsumer.class);
private InputStream is;
private final InputStream is;

public InputStreamConsumer(InputStream is) {
this.is = is;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,8 @@ public class SparkTempViewProvider implements TempViewProvider {

private static final Logger LOG = LoggerFactory.getLogger(SparkTempViewProvider.class);

private JavaSparkContext jsc;
private SQLContext sqlContext;
private final JavaSparkContext jsc;
private final SQLContext sqlContext;

public SparkTempViewProvider(String appName) {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ public abstract class AsyncClusteringService extends HoodieAsyncTableService {
private static final Logger LOG = LoggerFactory.getLogger(AsyncClusteringService.class);
private final int maxConcurrentClustering;
protected transient HoodieEngineContext context;
private transient BaseClusterer clusteringClient;
private final transient BaseClusterer clusteringClient;
vinothchandar marked this conversation as resolved.
Show resolved Hide resolved

public AsyncClusteringService(HoodieEngineContext context, BaseHoodieWriteClient writeClient) {
this(context, writeClient, false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ public abstract class AsyncCompactService extends HoodieAsyncTableService {
private static final Logger LOG = LoggerFactory.getLogger(AsyncCompactService.class);
private final int maxConcurrentCompaction;
protected transient HoodieEngineContext context;
private transient BaseCompactor compactor;
private final transient BaseCompactor compactor;

public AsyncCompactService(HoodieEngineContext context, BaseHoodieWriteClient client) {
this(context, client, false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,11 +57,11 @@ public abstract class HoodieAsyncService implements Serializable {
// Run in daemon mode
private final boolean runInDaemonMode;
// Queue to hold pending compaction/clustering instants
private transient BlockingQueue<String> pendingInstants = new LinkedBlockingQueue<>();
private final transient BlockingQueue<String> pendingInstants = new LinkedBlockingQueue<>();
vinothchandar marked this conversation as resolved.
Show resolved Hide resolved
// Mutex lock for synchronized access to pendingInstants queue
private transient ReentrantLock queueLock = new ReentrantLock();
private final transient ReentrantLock queueLock = new ReentrantLock();
// Condition instance to use with the queueLock
private transient Condition consumed = queueLock.newCondition();
private final transient Condition consumed = queueLock.newCondition();

protected HoodieAsyncService() {
this(false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,14 @@
*/
public class HoodieWriteCommitCallbackUtil {

private static ObjectMapper mapper = new ObjectMapper();
private static final ObjectMapper MAPPER = new ObjectMapper();

/**
* Convert data to json string format.
*/
public static String convertToJsonString(Object obj) {
try {
return mapper.writeValueAsString(obj);
return MAPPER.writeValueAsString(obj);
} catch (IOException e) {
throw new HoodieCommitCallbackException("Callback service convert data to json failed", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -259,14 +259,12 @@ public boolean isTrackingSuccessfulWrites() {

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WriteStatus {");
sb.append("fileId=").append(fileId);
sb.append(", writeStat=").append(stat);
sb.append(", globalError='").append(globalError).append('\'');
sb.append(", hasErrors='").append(hasErrors()).append('\'');
sb.append(", errorCount='").append(totalErrorRecords).append('\'');
sb.append(", errorPct='").append((100.0 * totalErrorRecords) / totalRecords).append('\'');
sb.append('}');
return sb.toString();
return "WriteStatus {" + "fileId=" + fileId
+ ", writeStat=" + stat
+ ", globalError='" + globalError + '\''
+ ", hasErrors='" + hasErrors() + '\''
+ ", errorCount='" + totalErrorRecords + '\''
+ ", errorPct='" + (100.0 * totalErrorRecords) / totalRecords + '\''
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -67,8 +67,8 @@ public class FileSystemBasedLockProvider implements LockProvider<String>, Serial
private final transient HoodieStorage storage;
private final transient StoragePath lockFile;
protected LockConfiguration lockConfiguration;
private SimpleDateFormat sdf;
private LockInfo lockInfo;
private final SimpleDateFormat sdf;
private final LockInfo lockInfo;
private String currentOwnerLockInfo;

public FileSystemBasedLockProvider(final LockConfiguration lockConfiguration, final StorageConfiguration<?> configuration) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ public class LockManager implements Serializable, AutoCloseable {
private final int maxRetries;
private final long maxWaitTimeInMs;
private final RetryHelper<Boolean, HoodieLockException> lockRetryHelper;
private transient HoodieLockMetrics metrics;
private final transient HoodieLockMetrics metrics;
private volatile LockProvider lockProvider;

public LockManager(HoodieWriteConfig writeConfig, HoodieStorage storage) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -524,7 +524,7 @@ public class HoodieIndexConfig extends HoodieConfig {
@Deprecated
public static final String DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH = SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE.defaultValue();

private EngineType engineType;
private final EngineType engineType;

/**
* Use Spark engine by default.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -835,7 +835,7 @@ public class HoodieWriteConfig extends HoodieConfig {
private HoodieStorageConfig storageConfig;
private HoodieTimeGeneratorConfig timeGeneratorConfig;
private HoodieIndexingConfig indexingConfig;
private EngineType engineType;
private final EngineType engineType;

/**
* @deprecated Use {@link #TBL_NAME} and its methods instead
Expand Down Expand Up @@ -2801,7 +2801,7 @@ public static class Builder {
private boolean isCleanConfigSet = false;
private boolean isArchivalConfigSet = false;
private boolean isClusteringConfigSet = false;
private boolean isOptimizeConfigSet = false;
private final boolean isOptimizeConfigSet = false;
private boolean isMetricsConfigSet = false;
private boolean isBootstrapConfigSet = false;
private boolean isMemoryConfigSet = false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,11 +90,9 @@ public int hashCode() {

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
sb.append(" fileId=").append(fileId);
sb.append(" minRecordKey=").append(minRecordKey);
sb.append(" maxRecordKey=").append(maxRecordKey);
sb.append('}');
return sb.toString();
return "BloomIndexFileInfo {" + " fileId=" + fileId
+ " minRecordKey=" + minRecordKey
+ " maxRecordKey=" + maxRecordKey
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.slf4j.LoggerFactory;

public class DefaultHBaseQPSResourceAllocator implements HBaseIndexQPSResourceAllocator {
private HoodieWriteConfig hoodieWriteConfig;
private final HoodieWriteConfig hoodieWriteConfig;
private static final Logger LOG = LoggerFactory.getLogger(DefaultHBaseQPSResourceAllocator.class);

public DefaultHBaseQPSResourceAllocator(HoodieWriteConfig hoodieWriteConfig) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,8 @@ public class HoodieMetrics {
private String conflictResolutionFailureCounterName = null;
private String compactionRequestedCounterName = null;
private String compactionCompletedCounterName = null;
private HoodieWriteConfig config;
private String tableName;
private final HoodieWriteConfig config;
private final String tableName;
private Timer rollbackTimer = null;
private Timer cleanTimer = null;
private Timer archiveTimer = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,12 +114,10 @@ public WriteOperationType getOperationType() {

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
sb.append("globalStat=").append(globalStat).append(", ");
sb.append("InputPartitionStat=").append(inputPartitionPathStatMap).append(", ");
sb.append("OutputPartitionStat=").append(outputPartitionPathStatMap).append(", ");
sb.append("operationType=").append(operationType);
sb.append('}');
return sb.toString();
return "WorkloadProfile {" + "globalStat=" + globalStat + ", "
+ "InputPartitionStat=" + inputPartitionPathStatMap + ", "
+ "OutputPartitionStat=" + outputPartitionPathStatMap + ", "
+ "operationType=" + operationType
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,9 +33,9 @@ public class WorkloadStat implements Serializable {

private long numUpdates = 0L;

private HashMap<String, Pair<String, Long>> insertLocationToCount;
private final HashMap<String, Pair<String, Long>> insertLocationToCount;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, for Java serialization and deserialization to work in Spark, these member variables cannot be final.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think for any class that already has final variables, it is safe to add additional final variables. Otherwise, we need to be careful around the Serializable.


private HashMap<String, Pair<String, Long>> updateLocationToCount;
private final HashMap<String, Pair<String, Long>> updateLocationToCount;

public WorkloadStat() {
insertLocationToCount = new HashMap<>();
Expand Down Expand Up @@ -86,10 +86,8 @@ public HashMap<String, Pair<String, Long>> getInsertLocationToCount() {

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadStat {");
sb.append("numInserts=").append(numInserts).append(", ");
sb.append("numUpdates=").append(numUpdates);
sb.append('}');
return sb.toString();
return "WorkloadStat {" + "numInserts=" + numInserts + ", "
+ "numUpdates=" + numUpdates
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ public class CleanPlanner<T, I, K, O> implements Serializable {
private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingLogCompactionOperations;
private final HoodieTable<T, I, K, O> hoodieTable;
private final HoodieWriteConfig config;
private transient HoodieEngineContext context;
private final transient HoodieEngineContext context;
private final List<String> savepointedTimestamps;
private Option<HoodieInstant> earliestCommitToRetain = Option.empty();

Expand Down Expand Up @@ -237,10 +237,7 @@ private boolean isAnySavepointDeleted(HoodieCleanMetadata cleanMetadata) {
// check for any savepointed removed in latest compared to previous saved list
List<String> removedSavepointedTimestamps = new ArrayList<>(savepointedTimestampsFromLastClean);
removedSavepointedTimestamps.removeAll(savepointedTimestamps);
if (removedSavepointedTimestamps.isEmpty()) {
return false;
}
return true;
return !removedSavepointedTimestamps.isEmpty();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,13 +151,16 @@ protected int getPlanVersion() {
* Transform {@link FileSlice} to {@link HoodieSliceInfo}.
*/
protected static List<HoodieSliceInfo> getFileSliceInfo(List<FileSlice> slices) {
return slices.stream().map(slice -> new HoodieSliceInfo().newBuilder()
.setPartitionPath(slice.getPartitionPath())
.setFileId(slice.getFileId())
.setDataFilePath(slice.getBaseFile().map(BaseFile::getPath).orElse(StringUtils.EMPTY_STRING))
.setDeltaFilePaths(slice.getLogFiles().map(f -> f.getPath().toString()).collect(Collectors.toList()))
.setBootstrapFilePath(slice.getBaseFile().map(bf -> bf.getBootstrapBaseFile().map(bbf -> bbf.getPath()).orElse(StringUtils.EMPTY_STRING)).orElse(StringUtils.EMPTY_STRING))
.build()).collect(Collectors.toList());
return slices.stream().map(slice -> {
new HoodieSliceInfo();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Redundant

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dont follow. checkstyle was failing due to bad indentation.

return HoodieSliceInfo.newBuilder()
.setPartitionPath(slice.getPartitionPath())
.setFileId(slice.getFileId())
.setDataFilePath(slice.getBaseFile().map(BaseFile::getPath).orElse(StringUtils.EMPTY_STRING))
.setDeltaFilePaths(slice.getLogFiles().map(f -> f.getPath().toString()).collect(Collectors.toList()))
.setBootstrapFilePath(slice.getBaseFile().map(bf -> bf.getBootstrapBaseFile().map(bbf -> bbf.getPath()).orElse(StringUtils.EMPTY_STRING)).orElse(StringUtils.EMPTY_STRING))
.build();
}).collect(Collectors.toList());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,12 +50,10 @@ public String getPartitionPath() {

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("BucketInfo {");
sb.append("bucketType=").append(bucketType).append(", ");
sb.append("fileIdPrefix=").append(fileIdPrefix).append(", ");
sb.append("partitionPath=").append(partitionPath);
sb.append('}');
return sb.toString();
return "BucketInfo {" + "bucketType=" + bucketType + ", "
+ "fileIdPrefix=" + fileIdPrefix + ", "
+ "partitionPath=" + partitionPath
+ '}';
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,10 +32,8 @@ public class InsertBucket implements Serializable {

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("InsertBucket {");
sb.append("bucketNumber=").append(bucketNumber).append(", ");
sb.append("weight=").append(weight);
sb.append('}');
return sb.toString();
return "InsertBucket {" + "bucketNumber=" + bucketNumber + ", "
+ "weight=" + weight
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,10 +32,8 @@ public class SmallFile implements Serializable {

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("SmallFile {");
sb.append("location=").append(location).append(", ");
sb.append("sizeBytes=").append(sizeBytes);
sb.append('}');
return sb.toString();
return "SmallFile {" + "location=" + location + ", "
+ "sizeBytes=" + sizeBytes
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public class RunCompactionActionExecutor<T> extends

private final HoodieCompactor compactor;
private final HoodieCompactionHandler compactionHandler;
private WriteOperationType operationType;
private final WriteOperationType operationType;

private final HoodieMetrics metrics;

Expand Down
Loading
Loading