Skip to content

Commit

Permalink
[Refactor] Use camelCase in thrift generated java sources (#4443)
Browse files Browse the repository at this point in the history
Use camelCase in thrift generated java sources to make us fe's code style is more unified
  • Loading branch information
yangzhg authored Aug 28, 2020
1 parent 0db9194 commit 3b7614e
Show file tree
Hide file tree
Showing 110 changed files with 968 additions and 947 deletions.
1 change: 1 addition & 0 deletions fe/fe-core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,7 @@ under the License.
<configuration>
<thriftExecutable>${doris.thridparty}/installed/bin/thrift</thriftExecutable>
<thriftSourceRoot>${doris.home}/gensrc/thrift</thriftSourceRoot>
<generator>java:fullcamel</generator>
</configuration>
<executions>
<execution>
Expand Down
12 changes: 6 additions & 6 deletions fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java
Original file line number Diff line number Diff line change
Expand Up @@ -574,7 +574,7 @@ public synchronized void handleFinishedReplica(AgentTask task, TTabletInfo finis
}
Preconditions.checkState(rollupIndex.getState() == IndexState.ROLLUP);

Preconditions.checkArgument(finishTabletInfo.getTablet_id() == rollupTabletId);
Preconditions.checkArgument(finishTabletInfo.getTabletId() == rollupTabletId);
Tablet rollupTablet = rollupIndex.getTablet(rollupTabletId);
if (rollupTablet == null) {
throw new MetaNotFoundException("Cannot find rollup tablet[" + rollupTabletId + "]");
Expand All @@ -591,15 +591,15 @@ public synchronized void handleFinishedReplica(AgentTask task, TTabletInfo finis
}

long version = finishTabletInfo.getVersion();
long versionHash = finishTabletInfo.getVersion_hash();
long dataSize = finishTabletInfo.getData_size();
long rowCount = finishTabletInfo.getRow_count();
long versionHash = finishTabletInfo.getVersionHash();
long dataSize = finishTabletInfo.getDataSize();
long rowCount = finishTabletInfo.getRowCount();
// yiguolei: not check version here because the replica's first version will be set by rollup job
// the version is not set now
rollupReplica.updateVersionInfo(version, versionHash, dataSize, rowCount);

if (finishTabletInfo.isSetPath_hash()) {
rollupReplica.setPathHash(finishTabletInfo.getPath_hash());
if (finishTabletInfo.isSetPathHash()) {
rollupReplica.setPathHash(finishTabletInfo.getPathHash());
}

setReplicaFinished(partitionId, rollupReplicaId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -567,7 +567,7 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo,
SchemaChangeTask schemaChangeTask = (SchemaChangeTask) task;

// check schema hash to avoid former schema change task try finishing current task
int finishTabletInfoSchemaHash = finishTabletInfo.getSchema_hash();
int finishTabletInfoSchemaHash = finishTabletInfo.getSchemaHash();
int taskSchemaHash = schemaChangeTask.getSchemaHash();
if (finishTabletInfoSchemaHash != taskSchemaHash) {
throw new MetaNotFoundException("Schema hash is not equal[" + finishTabletInfoSchemaHash + "-"
Expand Down Expand Up @@ -605,7 +605,7 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo,
}
Preconditions.checkState(materializedIndex.getState() == IndexState.SCHEMA_CHANGE);

Preconditions.checkArgument(finishTabletInfo.getTablet_id() == tabletId);
Preconditions.checkArgument(finishTabletInfo.getTabletId() == tabletId);
Tablet tablet = materializedIndex.getTablet(tabletId);
if (tablet == null) {
throw new MetaNotFoundException("Cannot find tablet[" + tabletId + "]");
Expand All @@ -618,13 +618,13 @@ public void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo,
// replica's state may be NORMAL(due to clone), so no need to check

long version = finishTabletInfo.getVersion();
long versionHash = finishTabletInfo.getVersion_hash();
long dataSize = finishTabletInfo.getData_size();
long rowCount = finishTabletInfo.getRow_count();
long versionHash = finishTabletInfo.getVersionHash();
long dataSize = finishTabletInfo.getDataSize();
long rowCount = finishTabletInfo.getRowCount();
// do not need check version > replica.getVersion, because the new replica's version is first set by sc
replica.updateVersionInfo(version, versionHash, dataSize, rowCount);
if (finishTabletInfo.isSetPath_hash()) {
replica.setPathHash(finishTabletInfo.getPath_hash());
if (finishTabletInfo.isSetPathHash()) {
replica.setPathHash(finishTabletInfo.getPathHash());
}
} finally {
db.writeUnlock();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ public TAnalyticWindowBoundary toThrift(Type windowType) {
TAnalyticWindowBoundary result = new TAnalyticWindowBoundary(type.toThrift());

if (type.isOffset() && windowType == Type.ROWS) {
result.setRows_offset_value(offsetValue.longValue());
result.setRowsOffsetValue(offsetValue.longValue());
}

// TODO: range windows need range_offset_predicate
Expand Down Expand Up @@ -297,13 +297,13 @@ public TAnalyticWindow toThrift() {
TAnalyticWindow result = new TAnalyticWindow(type_.toThrift());

if (leftBoundary_.getType() != BoundaryType.UNBOUNDED_PRECEDING) {
result.setWindow_start(leftBoundary_.toThrift(type_));
result.setWindowStart(leftBoundary_.toThrift(type_));
}

Preconditions.checkNotNull(rightBoundary_);

if (rightBoundary_.getType() != BoundaryType.UNBOUNDED_FOLLOWING) {
result.setWindow_end(rightBoundary_.toThrift(type_));
result.setWindowEnd(rightBoundary_.toThrift(type_));
}

return result;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,7 +382,7 @@ public static TQueryGlobals createQueryGlobals() {
TQueryGlobals queryGlobals = new TQueryGlobals();
Calendar currentDate = Calendar.getInstance();
String nowStr = formatter.format(currentDate.getTime());
queryGlobals.setNow_string(nowStr);
queryGlobals.setNowString(nowStr);
return queryGlobals;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,7 +169,7 @@ protected void toThrift(TExprNode msg) {
msg.node_type = TExprNodeType.ARITHMETIC_EXPR;
if (!type.isDecimal() && !type.isDecimalV2()) {
msg.setOpcode(op.getOpcode());
msg.setOutput_column(outputColumn);
msg.setOutputColumn(outputColumn);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -238,8 +238,8 @@ public String toSqlImpl() {
protected void toThrift(TExprNode msg) {
msg.node_type = TExprNodeType.BINARY_PRED;
msg.setOpcode(opcode);
msg.setVector_opcode(vectorOpcode);
msg.setChild_type(getChild(0).getType().getPrimitiveType().toThrift());
msg.setVectorOpcode(vectorOpcode);
msg.setChildType(getChild(0).getType().getPrimitiveType().toThrift());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ public TFunction toThrift() {
} else {
fn.setId(op_.thriftOp.ordinal());
}
fn.setAggregate_fn(new TAggregateFunction(intermediateType_.toThrift()));
fn.setAggregateFn(new TAggregateFunction(intermediateType_.toThrift()));
return fn;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,9 +162,9 @@ protected void treeToThriftHelper(TExpr container) {
protected void toThrift(TExprNode msg) {
msg.node_type = TExprNodeType.CAST_EXPR;
msg.setOpcode(opcode);
msg.setOutput_column(outputColumn);
msg.setOutputColumn(outputColumn);
if (type.isNativeType() && getChild(0).getType().isNativeType()) {
msg.setChild_type(getChild(0).getType().getPrimitiveType().toThrift());
msg.setChildType(getChild(0).getType().getPrimitiveType().toThrift());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -864,7 +864,7 @@ protected void treeToThriftHelper(TExpr container) {
if (fn != null) {
msg.setFn(fn.toThrift());
if (fn.hasVarArgs()) {
msg.setVararg_start_idx(fn.getNumArgs() - 1);
msg.setVarargStartIdx(fn.getNumArgs() - 1);
}
}
msg.output_scale = getOutputScale();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -274,7 +274,7 @@ protected void toThrift(TExprNode msg) {
if (isAggregate() || isAnalyticFnCall) {
msg.node_type = TExprNodeType.AGG_EXPR;
if (!isAnalyticFnCall) {
msg.setAgg_expr(new TAggregateExpr(isMergeAggFn));
msg.setAggExpr(new TAggregateExpr(isMergeAggFn));
}
} else {
msg.node_type = TExprNodeType.FUNCTION_CALL;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public static FunctionName createBuiltinName(String fn) {
}

public static FunctionName fromThrift(TFunctionName fnName) {
return new FunctionName(fnName.getDb_name(), fnName.getFunction_name());
return new FunctionName(fnName.getDbName(), fnName.getFunctionName());
}

@Override
Expand Down Expand Up @@ -176,8 +176,8 @@ private boolean isValidCharacter(char c) {

public TFunctionName toThrift() {
TFunctionName name = new TFunctionName(fn_);
name.setDb_name(db_);
name.setFunction_name(fn_);
name.setDbName(db_);
name.setFunctionName(fn_);
return name;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -230,7 +230,7 @@ protected void toThrift(TExprNode msg) {
msg.in_predicate = new TInPredicate(isNotIn);
msg.node_type = TExprNodeType.IN_PRED;
msg.setOpcode(opcode);
msg.setVector_opcode(vectorOpcode);
msg.setVectorOpcode(vectorOpcode);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -200,12 +200,12 @@ public String toSql() {
public TResultFileSinkOptions toSinkOptions() {
TResultFileSinkOptions sinkOptions = new TResultFileSinkOptions(filePath, fileFormatType);
if (isCsvFormat()) {
sinkOptions.setColumn_separator(columnSeparator);
sinkOptions.setLine_delimiter(lineDelimiter);
sinkOptions.setColumnSeparator(columnSeparator);
sinkOptions.setLineDelimiter(lineDelimiter);
}
sinkOptions.setMax_file_size_bytes(maxFileSizeBytes);
sinkOptions.setMaxFileSizeBytes(maxFileSizeBytes);
if (brokerDesc != null) {
sinkOptions.setBroker_properties(brokerDesc.getProperties());
sinkOptions.setBrokerProperties(brokerDesc.getProperties());
// broker_addresses of sinkOptions will be set in Coordinator.
// Because we need to choose the nearest broker with the result sink node.
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,7 +222,7 @@ public String toColumnLabel() {
protected void toThrift(TExprNode msg) {
msg.node_type = TExprNodeType.SLOT_REF;
msg.slot_ref = new TSlotRef(desc.getId().asInt(), desc.getParent().getId().asInt());
msg.setOutput_column(outputColumn);
msg.setOutputColumn(outputColumn);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ protected void toThrift(TExprNode msg) {
msg.node_type = TExprNodeType.TUPLE_IS_NULL_PRED;
msg.tuple_is_null_pred = new TTupleIsNullPredicate();
for (TupleId tid : tupleIds) {
msg.tuple_is_null_pred.addToTuple_ids(tid.asInt());
msg.tuple_is_null_pred.addToTupleIds(tid.asInt());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ public TUserIdentity toThrift() {
TUserIdentity tUserIdent = new TUserIdentity();
tUserIdent.setHost(host);
tUserIdent.setUsername(user);
tUserIdent.setIs_domain(isDomain);
tUserIdent.setIsDomain(isDomain);
return tUserIdent;
}

Expand Down
22 changes: 11 additions & 11 deletions fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java
Original file line number Diff line number Diff line change
Expand Up @@ -143,28 +143,28 @@ public String getLocalMetaInfoFilePath() {
public synchronized boolean finishTabletSnapshotTask(SnapshotTask task, TFinishTaskRequest request) {
Preconditions.checkState(task.getJobId() == jobId);

if (request.getTask_status().getStatus_code() != TStatusCode.OK) {
taskErrMsg.put(task.getSignature(), Joiner.on(",").join(request.getTask_status().getError_msgs()));
if (request.getTaskStatus().getStatusCode() != TStatusCode.OK) {
taskErrMsg.put(task.getSignature(), Joiner.on(",").join(request.getTaskStatus().getErrorMsgs()));
// snapshot task could not finish if status_code is OLAP_ERR_VERSION_ALREADY_MERGED,
// so cancel this job
if (request.getTask_status().getStatus_code() == TStatusCode.OLAP_ERR_VERSION_ALREADY_MERGED) {
if (request.getTaskStatus().getStatusCode() == TStatusCode.OLAP_ERR_VERSION_ALREADY_MERGED) {
status = new Status(ErrCode.OLAP_VERSION_ALREADY_MERGED, "make snapshot failed, version already merged");
cancelInternal();
}
return false;
}

Preconditions.checkState(request.isSetSnapshot_path());
Preconditions.checkState(request.isSetSnapshot_files());
Preconditions.checkState(request.isSetSnapshotPath());
Preconditions.checkState(request.isSetSnapshotFiles());
// snapshot path does not contains last 'tablet_id' and 'schema_hash' dir
// eg:
// /path/to/your/be/data/snapshot/20180410102311.0.86400/
// Full path will look like:
// /path/to/your/be/data/snapshot/20180410102311.0.86400/10006/352781111/
SnapshotInfo info = new SnapshotInfo(task.getDbId(), task.getTableId(), task.getPartitionId(),
task.getIndexId(), task.getTabletId(), task.getBackendId(),
task.getSchemaHash(), request.getSnapshot_path(),
request.getSnapshot_files());
task.getSchemaHash(), request.getSnapshotPath(),
request.getSnapshotFiles());

snapshotInfos.put(task.getTabletId(), info);
taskProgress.remove(task.getTabletId());
Expand All @@ -179,13 +179,13 @@ public synchronized boolean finishTabletSnapshotTask(SnapshotTask task, TFinishT
public synchronized boolean finishSnapshotUploadTask(UploadTask task, TFinishTaskRequest request) {
Preconditions.checkState(task.getJobId() == jobId);

if (request.getTask_status().getStatus_code() != TStatusCode.OK) {
taskErrMsg.put(task.getSignature(), Joiner.on(",").join(request.getTask_status().getError_msgs()));
if (request.getTaskStatus().getStatusCode() != TStatusCode.OK) {
taskErrMsg.put(task.getSignature(), Joiner.on(",").join(request.getTaskStatus().getErrorMsgs()));
return false;
}

Preconditions.checkState(request.isSetTablet_files());
Map<Long, List<String>> tabletFileMap = request.getTablet_files();
Preconditions.checkState(request.isSetTabletFiles());
Map<Long, List<String>> tabletFileMap = request.getTabletFiles();
if (tabletFileMap.isEmpty()) {
LOG.warn("upload snapshot files failed because nothing is uploaded. be: {}. {}",
task.getBackendId(), this);
Expand Down
12 changes: 6 additions & 6 deletions fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ public synchronized boolean finishTabletSnapshotTask(SnapshotTask task, TFinishT
return false;
}

Preconditions.checkState(request.isSetSnapshot_path());
Preconditions.checkState(request.isSetSnapshotPath());

// snapshot path does not contains last 'tablet_id' and 'schema_hash' dir
// eg:
Expand All @@ -185,7 +185,7 @@ public synchronized boolean finishTabletSnapshotTask(SnapshotTask task, TFinishT
// /path/to/your/be/data/snapshot/20180410102311.0/10006/352781111/
SnapshotInfo info = new SnapshotInfo(task.getDbId(), task.getTableId(), task.getPartitionId(),
task.getIndexId(), task.getTabletId(), task.getBackendId(),
task.getSchemaHash(), request.getSnapshot_path(), Lists.newArrayList());
task.getSchemaHash(), request.getSnapshotPath(), Lists.newArrayList());

snapshotInfos.put(task.getTabletId(), task.getBackendId(), info);
taskProgress.remove(task.getSignature());
Expand All @@ -206,9 +206,9 @@ public synchronized boolean finishTabletDownloadTask(DownloadTask task, TFinishT
return false;
}

Preconditions.checkState(request.isSetDownloaded_tablet_ids());
Preconditions.checkState(request.isSetDownloadedTabletIds());

for (Long tabletId : request.getDownloaded_tablet_ids()) {
for (Long tabletId : request.getDownloadedTabletIds()) {
SnapshotInfo info = snapshotInfos.get(tabletId, task.getBackendId());
if (info == null) {
LOG.error("failed to find snapshot infos of tablet {} in be {}, {}",
Expand Down Expand Up @@ -248,8 +248,8 @@ private boolean checkTaskStatus(AgentTask task, long jobId, TFinishTaskRequest r
Preconditions.checkState(jobId == this.jobId);
Preconditions.checkState(dbId == task.getDbId());

if (request.getTask_status().getStatus_code() != TStatusCode.OK) {
taskErrMsg.put(task.getSignature(), Joiner.on(",").join(request.getTask_status().getError_msgs()));
if (request.getTaskStatus().getStatusCode() != TStatusCode.OK) {
taskErrMsg.put(task.getSignature(), Joiner.on(",").join(request.getTaskStatus().getErrorMsgs()));
return true;
}
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -383,29 +383,29 @@ public String toSql(boolean ifNotExists) {
public TFunction toThrift() {
TFunction fn = super.toThrift();
TAggregateFunction aggFn = new TAggregateFunction();
aggFn.setIs_analytic_only_fn(isAnalyticFn && !isAggregateFn);
aggFn.setUpdate_fn_symbol(updateFnSymbol);
aggFn.setInit_fn_symbol(initFnSymbol);
aggFn.setIsAnalyticOnlyFn(isAnalyticFn && !isAggregateFn);
aggFn.setUpdateFnSymbol(updateFnSymbol);
aggFn.setInitFnSymbol(initFnSymbol);
if (serializeFnSymbol != null) {
aggFn.setSerialize_fn_symbol(serializeFnSymbol);
aggFn.setSerializeFnSymbol(serializeFnSymbol);
}
aggFn.setMerge_fn_symbol(mergeFnSymbol);
aggFn.setMergeFnSymbol(mergeFnSymbol);
if (getValueFnSymbol != null) {
aggFn.setGet_value_fn_symbol(getValueFnSymbol);
aggFn.setGetValueFnSymbol(getValueFnSymbol);
}
if (removeFnSymbol != null) {
aggFn.setRemove_fn_symbol(removeFnSymbol);
aggFn.setRemoveFnSymbol(removeFnSymbol);
}
if (finalizeFnSymbol != null) {
aggFn.setFinalize_fn_symbol(finalizeFnSymbol);
aggFn.setFinalizeFnSymbol(finalizeFnSymbol);
}
if (intermediateType != null) {
aggFn.setIntermediate_type(intermediateType.toThrift());
aggFn.setIntermediateType(intermediateType.toThrift());
} else {
aggFn.setIntermediate_type(getReturnType().toThrift());
aggFn.setIntermediateType(getReturnType().toThrift());
}
// agg_fn.setIgnores_distinct(ignoresDistinct);
fn.setAggregate_fn(aggFn);
fn.setAggregateFn(aggFn);
return fn;
}

Expand Down
Loading

0 comments on commit 3b7614e

Please sign in to comment.