twalthr commented on a change in pull request #16213:
URL: https://github.com/apache/flink/pull/16213#discussion_r655372351
##########
File path:
flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/EqualiserCodeGeneratorTest.java
##########
@@ -81,6 +83,22 @@ public void testTimestamp() {
assertBoolean(equaliser, func, fromEpochMillis(1024),
fromEpochMillis(1025), false);
}
+ @Test
+ public void testManyFields() {
+ final LogicalType[] fieldTypes =
+ IntStream.range(0, 999)
+ .mapToObj(i -> new VarCharType())
+ .toArray(LogicalType[]::new);
+
+ try {
+ new EqualiserCodeGenerator(fieldTypes)
Review comment:
can we also test the result to be on the safe side?
##########
File path:
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/EqualiserCodeGenerator.scala
##########
@@ -121,17 +64,97 @@ class EqualiserCodeGenerator(fieldTypes:
Array[LogicalType]) {
public boolean equals($ROW_DATA $LEFT_INPUT, $ROW_DATA $RIGHT_INPUT)
{
if ($LEFT_INPUT instanceof $BINARY_ROW && $RIGHT_INPUT instanceof
$BINARY_ROW) {
return $LEFT_INPUT.equals($RIGHT_INPUT);
- } else {
- $header
- ${ctx.reuseLocalVariableCode()}
- ${codes.mkString("\n")}
- return true;
}
+
+ if ($LEFT_INPUT.getRowKind() != $RIGHT_INPUT.getRowKind()) {
+ return false;
+ }
+
+ boolean result = true;
+ ${equalsMethodCalls.mkString("\n")}
+ return result;
}
+
+ ${equalsMethodCodes.mkString("\n")}
}
""".stripMargin
- new GeneratedRecordEqualiser(className, functionCode,
ctx.references.toArray)
+ new GeneratedRecordEqualiser(className, classCode, ctx.references.toArray)
+ }
+
+ private def getEqualsMethodName(idx: Int) = s"""equalsAtIndex$idx"""
+
+ private def generateEqualsMethod(ctx: CodeGeneratorContext, idx: Int):
String = {
+ val methodName = getEqualsMethodName(idx)
+ ctx.startNewLocalVariableStatement(methodName)
+
+ val Seq(leftNullTerm, rightNullTerm) = ctx.addReusableLocalVariables(
+ ("boolean", "isNullLeft"),
+ ("boolean", "isNullRight")
+ )
+
+ val fieldType = fieldTypes(idx)
+ val fieldTypeTerm = primitiveTypeTermForType(fieldType)
+ val Seq(leftFieldTerm, rightFieldTerm) = ctx.addReusableLocalVariables(
+ (fieldTypeTerm, "leftField"),
+ (fieldTypeTerm, "rightField")
+ )
+
+ val leftReadCode = rowFieldReadAccess(ctx, idx, LEFT_INPUT, fieldType)
+ val rightReadCode = rowFieldReadAccess(ctx, idx, RIGHT_INPUT, fieldType)
+
+ val (equalsCode, equalsResult) = generateEqualsCode(ctx, fieldType,
+ leftFieldTerm, rightFieldTerm, leftNullTerm, rightNullTerm)
+
+ s"""
+ |private boolean $methodName($ROW_DATA $LEFT_INPUT, $ROW_DATA
$RIGHT_INPUT) {
+ | ${ctx.reuseLocalVariableCode(methodName)}
+ |
+ | $leftNullTerm = $LEFT_INPUT.isNullAt($idx);
+ | $rightNullTerm = $RIGHT_INPUT.isNullAt($idx);
+ | if ($leftNullTerm && $rightNullTerm) {
+ | return true;
+ | }
+ |
+ | if ($leftNullTerm|| $rightNullTerm) {
+ | return false;
+ | }
Review comment:
we could perform the null checks until this location in a loop? This
should also improve performance because we don't need to call the equals
methods anymore.
##########
File path:
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/EqualiserCodeGenerator.scala
##########
@@ -121,17 +64,97 @@ class EqualiserCodeGenerator(fieldTypes:
Array[LogicalType]) {
public boolean equals($ROW_DATA $LEFT_INPUT, $ROW_DATA $RIGHT_INPUT)
{
if ($LEFT_INPUT instanceof $BINARY_ROW && $RIGHT_INPUT instanceof
$BINARY_ROW) {
return $LEFT_INPUT.equals($RIGHT_INPUT);
- } else {
- $header
- ${ctx.reuseLocalVariableCode()}
- ${codes.mkString("\n")}
- return true;
}
+
+ if ($LEFT_INPUT.getRowKind() != $RIGHT_INPUT.getRowKind()) {
+ return false;
+ }
+
+ boolean result = true;
+ ${equalsMethodCalls.mkString("\n")}
+ return result;
}
+
+ ${equalsMethodCodes.mkString("\n")}
}
""".stripMargin
- new GeneratedRecordEqualiser(className, functionCode,
ctx.references.toArray)
+ new GeneratedRecordEqualiser(className, classCode, ctx.references.toArray)
+ }
+
+ private def getEqualsMethodName(idx: Int) = s"""equalsAtIndex$idx"""
+
+ private def generateEqualsMethod(ctx: CodeGeneratorContext, idx: Int):
String = {
+ val methodName = getEqualsMethodName(idx)
+ ctx.startNewLocalVariableStatement(methodName)
+
+ val Seq(leftNullTerm, rightNullTerm) = ctx.addReusableLocalVariables(
+ ("boolean", "isNullLeft"),
+ ("boolean", "isNullRight")
+ )
+
+ val fieldType = fieldTypes(idx)
+ val fieldTypeTerm = primitiveTypeTermForType(fieldType)
+ val Seq(leftFieldTerm, rightFieldTerm) = ctx.addReusableLocalVariables(
+ (fieldTypeTerm, "leftField"),
+ (fieldTypeTerm, "rightField")
+ )
+
+ val leftReadCode = rowFieldReadAccess(ctx, idx, LEFT_INPUT, fieldType)
+ val rightReadCode = rowFieldReadAccess(ctx, idx, RIGHT_INPUT, fieldType)
+
+ val (equalsCode, equalsResult) = generateEqualsCode(ctx, fieldType,
+ leftFieldTerm, rightFieldTerm, leftNullTerm, rightNullTerm)
+
+ s"""
+ |private boolean $methodName($ROW_DATA $LEFT_INPUT, $ROW_DATA
$RIGHT_INPUT) {
+ | ${ctx.reuseLocalVariableCode(methodName)}
+ |
+ | $leftNullTerm = $LEFT_INPUT.isNullAt($idx);
+ | $rightNullTerm = $RIGHT_INPUT.isNullAt($idx);
+ | if ($leftNullTerm && $rightNullTerm) {
+ | return true;
+ | }
+ |
+ | if ($leftNullTerm|| $rightNullTerm) {
Review comment:
missing space before `||`
##########
File path:
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/EqualiserCodeGenerator.scala
##########
@@ -121,17 +64,97 @@ class EqualiserCodeGenerator(fieldTypes:
Array[LogicalType]) {
public boolean equals($ROW_DATA $LEFT_INPUT, $ROW_DATA $RIGHT_INPUT)
{
if ($LEFT_INPUT instanceof $BINARY_ROW && $RIGHT_INPUT instanceof
$BINARY_ROW) {
return $LEFT_INPUT.equals($RIGHT_INPUT);
- } else {
- $header
- ${ctx.reuseLocalVariableCode()}
- ${codes.mkString("\n")}
- return true;
}
+
+ if ($LEFT_INPUT.getRowKind() != $RIGHT_INPUT.getRowKind()) {
+ return false;
+ }
+
+ boolean result = true;
+ ${equalsMethodCalls.mkString("\n")}
+ return result;
}
+
+ ${equalsMethodCodes.mkString("\n")}
}
""".stripMargin
- new GeneratedRecordEqualiser(className, functionCode,
ctx.references.toArray)
+ new GeneratedRecordEqualiser(className, classCode, ctx.references.toArray)
+ }
+
+ private def getEqualsMethodName(idx: Int) = s"""equalsAtIndex$idx"""
+
+ private def generateEqualsMethod(ctx: CodeGeneratorContext, idx: Int):
String = {
+ val methodName = getEqualsMethodName(idx)
+ ctx.startNewLocalVariableStatement(methodName)
+
+ val Seq(leftNullTerm, rightNullTerm) = ctx.addReusableLocalVariables(
+ ("boolean", "isNullLeft"),
+ ("boolean", "isNullRight")
+ )
+
+ val fieldType = fieldTypes(idx)
+ val fieldTypeTerm = primitiveTypeTermForType(fieldType)
+ val Seq(leftFieldTerm, rightFieldTerm) = ctx.addReusableLocalVariables(
+ (fieldTypeTerm, "leftField"),
+ (fieldTypeTerm, "rightField")
+ )
+
+ val leftReadCode = rowFieldReadAccess(ctx, idx, LEFT_INPUT, fieldType)
+ val rightReadCode = rowFieldReadAccess(ctx, idx, RIGHT_INPUT, fieldType)
+
+ val (equalsCode, equalsResult) = generateEqualsCode(ctx, fieldType,
+ leftFieldTerm, rightFieldTerm, leftNullTerm, rightNullTerm)
+
+ s"""
+ |private boolean $methodName($ROW_DATA $LEFT_INPUT, $ROW_DATA
$RIGHT_INPUT) {
+ | ${ctx.reuseLocalVariableCode(methodName)}
+ |
+ | $leftNullTerm = $LEFT_INPUT.isNullAt($idx);
+ | $rightNullTerm = $RIGHT_INPUT.isNullAt($idx);
+ | if ($leftNullTerm && $rightNullTerm) {
+ | return true;
+ | }
+ |
+ | if ($leftNullTerm|| $rightNullTerm) {
+ | return false;
+ | }
+ |
+ | $leftFieldTerm = $leftReadCode;
+ | $rightFieldTerm = $rightReadCode;
+ | $equalsCode
+ |
+ | return $equalsResult;
+ |}
+ """.stripMargin
+ }
+
+ private def generateEqualsCode(ctx: CodeGeneratorContext, fieldType:
LogicalType,
+ leftFieldTerm: String, rightFieldTerm: String,
+ leftNullTerm: String, rightNullTerm: String) = {
+ if (isInternalPrimitive(fieldType)) {
Review comment:
preserve original TODO
##########
File path:
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/EqualiserCodeGenerator.scala
##########
@@ -121,17 +64,97 @@ class EqualiserCodeGenerator(fieldTypes:
Array[LogicalType]) {
public boolean equals($ROW_DATA $LEFT_INPUT, $ROW_DATA $RIGHT_INPUT)
{
if ($LEFT_INPUT instanceof $BINARY_ROW && $RIGHT_INPUT instanceof
$BINARY_ROW) {
return $LEFT_INPUT.equals($RIGHT_INPUT);
- } else {
- $header
- ${ctx.reuseLocalVariableCode()}
- ${codes.mkString("\n")}
- return true;
}
+
+ if ($LEFT_INPUT.getRowKind() != $RIGHT_INPUT.getRowKind()) {
+ return false;
+ }
+
+ boolean result = true;
+ ${equalsMethodCalls.mkString("\n")}
+ return result;
}
+
+ ${equalsMethodCodes.mkString("\n")}
}
""".stripMargin
- new GeneratedRecordEqualiser(className, functionCode,
ctx.references.toArray)
+ new GeneratedRecordEqualiser(className, classCode, ctx.references.toArray)
+ }
+
+ private def getEqualsMethodName(idx: Int) = s"""equalsAtIndex$idx"""
+
+ private def generateEqualsMethod(ctx: CodeGeneratorContext, idx: Int):
String = {
+ val methodName = getEqualsMethodName(idx)
+ ctx.startNewLocalVariableStatement(methodName)
+
+ val Seq(leftNullTerm, rightNullTerm) = ctx.addReusableLocalVariables(
+ ("boolean", "isNullLeft"),
+ ("boolean", "isNullRight")
+ )
+
+ val fieldType = fieldTypes(idx)
+ val fieldTypeTerm = primitiveTypeTermForType(fieldType)
+ val Seq(leftFieldTerm, rightFieldTerm) = ctx.addReusableLocalVariables(
+ (fieldTypeTerm, "leftField"),
+ (fieldTypeTerm, "rightField")
+ )
+
+ val leftReadCode = rowFieldReadAccess(ctx, idx, LEFT_INPUT, fieldType)
+ val rightReadCode = rowFieldReadAccess(ctx, idx, RIGHT_INPUT, fieldType)
+
+ val (equalsCode, equalsResult) = generateEqualsCode(ctx, fieldType,
+ leftFieldTerm, rightFieldTerm, leftNullTerm, rightNullTerm)
+
+ s"""
+ |private boolean $methodName($ROW_DATA $LEFT_INPUT, $ROW_DATA
$RIGHT_INPUT) {
+ | ${ctx.reuseLocalVariableCode(methodName)}
+ |
+ | $leftNullTerm = $LEFT_INPUT.isNullAt($idx);
+ | $rightNullTerm = $RIGHT_INPUT.isNullAt($idx);
+ | if ($leftNullTerm && $rightNullTerm) {
+ | return true;
+ | }
+ |
+ | if ($leftNullTerm|| $rightNullTerm) {
+ | return false;
+ | }
Review comment:
You are right. I just saw the similarities without checking deeper.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]