Skip to content
Open
Show file tree
Hide file tree
Changes from all 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
16 changes: 14 additions & 2 deletions fe/fe-catalog/src/main/java/org/apache/doris/catalog/Function.java
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,8 @@ public enum BinaryType {
protected String runtimeVersion;
@SerializedName("fc")
protected String functionCode;
@SerializedName("det")
protected boolean deterministic = false;
Comment thread
linrrzqqq marked this conversation as resolved.

// Only used for serialization
protected Function() {
Expand Down Expand Up @@ -174,6 +176,7 @@ public Function(Function other) {
this.expirationTime = other.expirationTime;
this.runtimeVersion = other.runtimeVersion;
this.functionCode = other.functionCode;
this.deterministic = other.deterministic;
}

public Function clone() {
Expand Down Expand Up @@ -301,6 +304,14 @@ public void setFunctionCode(String functionCode) {
this.functionCode = functionCode;
}

public boolean isDeterministic() {
return deterministic;
}

public void setDeterministic(boolean deterministic) {
this.deterministic = deterministic;
}

// TODO(cmy): Currently we judge whether it is UDF by wheter the 'location' is set.
// Maybe we should use a separate variable to identify,
// but additional variables need to modify the persistence information.
Expand Down Expand Up @@ -401,7 +412,8 @@ public boolean equals(Object o) {
}
Function function = (Function) o;
return id == function.id && hasVarArgs == function.hasVarArgs && userVisible == function.userVisible
&& vectorized == function.vectorized && Objects.equals(name, function.name)
&& vectorized == function.vectorized && deterministic == function.deterministic
&& Objects.equals(name, function.name)
&& Objects.equals(retType, function.retType) && Arrays.equals(argTypes,
function.argTypes) && Objects.equals(location, function.location)
&& binaryType == function.binaryType && nullableMode == function.nullableMode && Objects.equals(
Expand All @@ -411,7 +423,7 @@ public boolean equals(Object o) {
@Override
public int hashCode() {
int result = Objects.hash(id, name, retType, hasVarArgs, userVisible, location, binaryType, nullableMode,
vectorized, checksum);
vectorized, checksum, deterministic);
result = 31 * result + Arrays.hashCode(argTypes);
return result;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,13 @@ public static String toSql(ScalarFunction fn, boolean ifNotExists) {
.append("\"" + (fn.getLocation() == null ? "" : fn.getLocation().toString()) + "\"");
boolean isReturnNull = fn.getNullableMode() == NullableMode.ALWAYS_NULLABLE;
sb.append(",\n \"ALWAYS_NULLABLE\"=").append("\"" + isReturnNull + "\"");
sb.append(",\n \"DETERMINISTIC\"=").append("\"" + fn.isDeterministic() + "\"");
} else if (fn.getBinaryType() == Function.BinaryType.PYTHON_UDF) {
sb.append(",\n \"FILE\"=")
.append("\"" + (fn.getLocation() == null ? "" : fn.getLocation().toString()) + "\"");
boolean isReturnNull = fn.getNullableMode() == NullableMode.ALWAYS_NULLABLE;
sb.append(",\n \"ALWAYS_NULLABLE\"=").append("\"" + isReturnNull + "\"");
sb.append(",\n \"DETERMINISTIC\"=").append("\"" + fn.isDeterministic() + "\"");
} else {
sb.append(",\n \"OBJECT_FILE\"=")
.append("\"" + (fn.getLocation() == null ? "" : fn.getLocation().toString()) + "\"");
Expand Down Expand Up @@ -125,6 +132,13 @@ public static String toSql(AggregateFunction fn, boolean ifNotExists) {
.append("\"" + (fn.getLocation() == null ? "" : fn.getLocation().toString()) + "\",");
boolean isReturnNull = fn.getNullableMode() == NullableMode.ALWAYS_NULLABLE;
sb.append("\n \"ALWAYS_NULLABLE\"=").append("\"" + isReturnNull + "\",");
sb.append("\n \"DETERMINISTIC\"=").append("\"" + fn.isDeterministic() + "\",");
} else if (fn.getBinaryType() == Function.BinaryType.PYTHON_UDF) {
sb.append("\n \"FILE\"=")
.append("\"" + (fn.getLocation() == null ? "" : fn.getLocation().toString()) + "\",");
boolean isReturnNull = fn.getNullableMode() == NullableMode.ALWAYS_NULLABLE;
sb.append("\n \"ALWAYS_NULLABLE\"=").append("\"" + isReturnNull + "\",");
sb.append("\n \"DETERMINISTIC\"=").append("\"" + fn.isDeterministic() + "\",");
} else {
sb.append("\n \"OBJECT_FILE\"=")
.append("\"" + (fn.getLocation() == null ? "" : fn.getLocation().toString()) + "\",");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ public class JavaUdaf extends AggregateFunction implements ExplicitlyCastableSig
private final String checkSum;
private final boolean isStaticLoad;
private final long expirationTime;
private final boolean deterministic;

/**
* Constructor of UDAF
Expand All @@ -72,7 +73,8 @@ public JavaUdaf(String name, long functionId, String dbName, Function.BinaryType
String objectFile, String symbol,
String initFn, String updateFn, String mergeFn,
String serializeFn, String finalizeFn, String getValueFn, String removeFn,
boolean isDistinct, String checkSum, boolean isStaticLoad, long expirationTime, Expression... args) {
boolean isDistinct, String checkSum, boolean isStaticLoad, long expirationTime,
boolean deterministic, Expression... args) {
super(name, isDistinct, args);
this.dbName = dbName;
this.functionId = functionId;
Expand All @@ -92,6 +94,7 @@ public JavaUdaf(String name, long functionId, String dbName, Function.BinaryType
this.checkSum = checkSum;
this.isStaticLoad = isStaticLoad;
this.expirationTime = expirationTime;
this.deterministic = deterministic;
}

@Override
Expand All @@ -114,6 +117,11 @@ public NullableMode getNullableMode() {
return nullableMode;
}

@Override
public boolean isDeterministic() {
return deterministic;
}

/**
* withChildren.
*/
Expand All @@ -122,7 +130,8 @@ public JavaUdaf withDistinctAndChildren(boolean isDistinct, List<Expression> chi
Preconditions.checkArgument(children.size() == this.children.size());
return new JavaUdaf(getName(), functionId, dbName, binaryType, signature, intermediateType, nullableMode,
objectFile, symbol, initFn, updateFn, mergeFn, serializeFn, finalizeFn, getValueFn, removeFn,
isDistinct, checkSum, isStaticLoad, expirationTime, children.toArray(new Expression[0]));
isDistinct, checkSum, isStaticLoad, expirationTime, deterministic,
children.toArray(new Expression[0]));
}

/**
Expand Down Expand Up @@ -165,6 +174,7 @@ public static void translateToNereidsFunction(String dbName, org.apache.doris.ca
aggregate.getChecksum(),
aggregate.isStaticLoad(),
aggregate.getExpirationTime(),
aggregate.isDeterministic(),
arguments);

JavaUdafBuilder builder = new JavaUdafBuilder(udaf);
Expand Down Expand Up @@ -201,6 +211,7 @@ public Function getCatalogFunction() {
expr.setId(functionId);
expr.setStaticLoad(isStaticLoad);
expr.setExpirationTime(expirationTime);
expr.setDeterministic(deterministic);
return expr;
} catch (Exception e) {
throw new AnalysisException(e.getMessage(), e.getCause());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,14 +56,15 @@ public class JavaUdf extends ScalarFunction implements ExplicitlyCastableSignatu
private final String checkSum;
private final boolean isStaticLoad;
private final long expirationTime;
private final boolean deterministic;

/**
* Constructor of UDF
*/
public JavaUdf(String name, long functionId, String dbName, Function.BinaryType binaryType,
FunctionSignature signature,
NullableMode nullableMode, String objectFile, String symbol, String prepareFn, String closeFn,
String checkSum, boolean isStaticLoad, long expirationTime, Expression... args) {
String checkSum, boolean isStaticLoad, long expirationTime, boolean deterministic, Expression... args) {
super(name, args);
this.dbName = dbName;
this.functionId = functionId;
Expand All @@ -77,6 +78,7 @@ public JavaUdf(String name, long functionId, String dbName, Function.BinaryType
this.checkSum = checkSum;
this.isStaticLoad = isStaticLoad;
this.expirationTime = expirationTime;
this.deterministic = deterministic;
}

@Override
Expand All @@ -99,14 +101,19 @@ public NullableMode getNullableMode() {
return nullableMode;
}

@Override
public boolean isDeterministic() {
Comment thread
linrrzqqq marked this conversation as resolved.
return deterministic;
}

/**
* withChildren.
*/
@Override
public JavaUdf withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == this.children.size());
return new JavaUdf(getName(), functionId, dbName, binaryType, signature, nullableMode,
objectFile, symbol, prepareFn, closeFn, checkSum, isStaticLoad, expirationTime,
objectFile, symbol, prepareFn, closeFn, checkSum, isStaticLoad, expirationTime, deterministic,
children.toArray(new Expression[0]));
}

Expand Down Expand Up @@ -135,7 +142,7 @@ public static void translateToNereidsFunction(String dbName, org.apache.doris.ca
scalar.getSymbolName(),
scalar.getPrepareFnSymbol(),
scalar.getCloseFnSymbol(),
scalar.getChecksum(), scalar.isStaticLoad(), scalar.getExpirationTime(),
scalar.getChecksum(), scalar.isStaticLoad(), scalar.getExpirationTime(), scalar.isDeterministic(),
arguments);

JavaUdfBuilder builder = new JavaUdfBuilder(udf);
Expand Down Expand Up @@ -166,6 +173,7 @@ public Function getCatalogFunction() {
expr.setId(functionId);
expr.setStaticLoad(isStaticLoad);
expr.setExpirationTime(expirationTime);
expr.setDeterministic(deterministic);
return expr;
} catch (Exception e) {
throw new AnalysisException(e.getMessage(), e.getCause());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,14 +56,15 @@ public class JavaUdtf extends TableGeneratingFunction implements ExplicitlyCasta
private final String checkSum;
private final boolean isStaticLoad;
private final long expirationTime;
private final boolean deterministic;

/**
* Constructor of UDTF
*/
public JavaUdtf(String name, long functionId, String dbName, Function.BinaryType binaryType,
FunctionSignature signature,
NullableMode nullableMode, String objectFile, String symbol, String prepareFn, String closeFn,
String checkSum, boolean isStaticLoad, long expirationTime, Expression... args) {
String checkSum, boolean isStaticLoad, long expirationTime, boolean deterministic, Expression... args) {
super(name, args);
this.dbName = dbName;
this.functionId = functionId;
Expand All @@ -77,6 +78,7 @@ public JavaUdtf(String name, long functionId, String dbName, Function.BinaryType
this.checkSum = checkSum;
this.isStaticLoad = isStaticLoad;
this.expirationTime = expirationTime;
this.deterministic = deterministic;
}

/**
Expand All @@ -86,7 +88,7 @@ public JavaUdtf(String name, long functionId, String dbName, Function.BinaryType
public JavaUdtf withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == this.children.size());
return new JavaUdtf(getName(), functionId, dbName, binaryType, signature, nullableMode,
objectFile, symbol, prepareFn, closeFn, checkSum, isStaticLoad, expirationTime,
objectFile, symbol, prepareFn, closeFn, checkSum, isStaticLoad, expirationTime, deterministic,
children.toArray(new Expression[0]));
}

Expand All @@ -95,6 +97,11 @@ public List<FunctionSignature> getSignatures() {
return ImmutableList.of(signature);
}

@Override
public boolean isDeterministic() {
return deterministic;
}

@Override
public boolean hasVarArguments() {
return signature.hasVarArgs;
Expand Down Expand Up @@ -125,6 +132,7 @@ public Function getCatalogFunction() {
expr.setStaticLoad(isStaticLoad);
expr.setExpirationTime(expirationTime);
expr.setUDTFunction(true);
expr.setDeterministic(deterministic);
return expr;
} catch (Exception e) {
throw new AnalysisException(e.getMessage(), e.getCause());
Expand Down Expand Up @@ -159,6 +167,7 @@ public static void translateToNereidsFunction(String dbName, org.apache.doris.ca
scalar.getChecksum(),
scalar.isStaticLoad(),
scalar.getExpirationTime(),
scalar.isDeterministic(),
arguments);

JavaUdtfBuilder builder = new JavaUdtfBuilder(udf);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ public class PythonUdaf extends AggregateFunction implements ExplicitlyCastableS
private final long expirationTime;
private final String runtimeVersion;
private final String functionCode;
private final boolean deterministic;

/**
* Constructor of UDAF
Expand All @@ -75,7 +76,7 @@ public PythonUdaf(String name, long functionId, String dbName, Function.BinaryTy
String initFn, String updateFn, String mergeFn,
String serializeFn, String finalizeFn, String getValueFn, String removeFn,
boolean isDistinct, String checkSum, boolean isStaticLoad, long expirationTime,
String runtimeVersion, String functionCode, Expression... args) {
String runtimeVersion, String functionCode, boolean deterministic, Expression... args) {
super(name, isDistinct, args);
this.dbName = dbName;
this.functionId = functionId;
Expand All @@ -97,6 +98,7 @@ public PythonUdaf(String name, long functionId, String dbName, Function.BinaryTy
this.expirationTime = expirationTime;
this.runtimeVersion = runtimeVersion;
this.functionCode = functionCode;
this.deterministic = deterministic;
}

@Override
Expand All @@ -119,6 +121,11 @@ public NullableMode getNullableMode() {
return nullableMode;
}

@Override
public boolean isDeterministic() {
return deterministic;
}

/**
* withChildren.
*/
Expand All @@ -127,7 +134,7 @@ public PythonUdaf withDistinctAndChildren(boolean isDistinct, List<Expression> c
Preconditions.checkArgument(children.size() == this.children.size());
return new PythonUdaf(getName(), functionId, dbName, binaryType, signature, intermediateType, nullableMode,
objectFile, symbol, initFn, updateFn, mergeFn, serializeFn, finalizeFn, getValueFn, removeFn,
isDistinct, checkSum, isStaticLoad, expirationTime, runtimeVersion, functionCode,
isDistinct, checkSum, isStaticLoad, expirationTime, runtimeVersion, functionCode, deterministic,
children.toArray(new Expression[0]));
}

Expand Down Expand Up @@ -173,6 +180,7 @@ public static void translateToNereidsFunction(String dbName, org.apache.doris.ca
aggregate.getExpirationTime(),
aggregate.getRuntimeVersion(),
aggregate.getFunctionCode(),
aggregate.isDeterministic(),
arguments);

PythonUdafBuilder builder = new PythonUdafBuilder(udaf);
Expand Down Expand Up @@ -211,6 +219,7 @@ public Function getCatalogFunction() {
expr.setExpirationTime(expirationTime);
expr.setRuntimeVersion(runtimeVersion);
expr.setFunctionCode(functionCode);
expr.setDeterministic(deterministic);
return expr;
} catch (Exception e) {
throw new AnalysisException(e.getMessage(), e.getCause());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ public class PythonUdf extends ScalarFunction implements ExplicitlyCastableSigna
private final long expirationTime;
private final String runtimeVersion;
private final String functionCode;
private final boolean deterministic;

/**
* Constructor of UDF
Expand All @@ -66,7 +67,7 @@ public PythonUdf(String name, long functionId, String dbName, Function.BinaryTyp
FunctionSignature signature,
NullableMode nullableMode, String objectFile, String symbol, String prepareFn, String closeFn,
String checkSum, boolean isStaticLoad, long expirationTime,
String runtimeVersion, String functionCode, Expression... args) {
String runtimeVersion, String functionCode, boolean deterministic, Expression... args) {
super(name, args);
this.dbName = dbName;
this.functionId = functionId;
Expand All @@ -82,6 +83,7 @@ public PythonUdf(String name, long functionId, String dbName, Function.BinaryTyp
this.expirationTime = expirationTime;
this.runtimeVersion = runtimeVersion;
this.functionCode = functionCode;
this.deterministic = deterministic;
}

@Override
Expand All @@ -104,6 +106,11 @@ public NullableMode getNullableMode() {
return nullableMode;
}

@Override
public boolean isDeterministic() {
Comment thread
linrrzqqq marked this conversation as resolved.
return deterministic;
}

/**
* withChildren.
*/
Expand All @@ -112,7 +119,7 @@ public PythonUdf withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == this.children.size());
return new PythonUdf(getName(), functionId, dbName, binaryType, signature, nullableMode,
objectFile, symbol, prepareFn, closeFn, checkSum, isStaticLoad, expirationTime,
runtimeVersion, functionCode, children.toArray(new Expression[0]));
runtimeVersion, functionCode, deterministic, children.toArray(new Expression[0]));
}

/**
Expand Down Expand Up @@ -143,6 +150,7 @@ public static void translateToNereidsFunction(String dbName, org.apache.doris.ca
scalar.getChecksum(), scalar.isStaticLoad(), scalar.getExpirationTime(),
scalar.getRuntimeVersion(),
scalar.getFunctionCode(),
scalar.isDeterministic(),
arguments);

PythonUdfBuilder builder = new PythonUdfBuilder(udf);
Expand Down Expand Up @@ -175,6 +183,7 @@ public Function getCatalogFunction() {
expr.setExpirationTime(expirationTime);
expr.setRuntimeVersion(runtimeVersion);
expr.setFunctionCode(functionCode);
expr.setDeterministic(deterministic);
return expr;
} catch (Exception e) {
throw new AnalysisException(e.getMessage(), e.getCause());
Expand Down
Loading
Loading