diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala index fde17963bfd0..734535c2e14c 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala @@ -1270,14 +1270,17 @@ class Column private[sql] (private[sql] val expr: proto.Expression) extends Logg private[sql] object Column { - def apply(name: String): Column = Column { builder => + def apply(name: String): Column = Column(name, None) + + def apply(name: String, planId: Option[Long]): Column = Column { builder => name match { case "*" => builder.getUnresolvedStarBuilder case _ if name.endsWith(".*") => builder.getUnresolvedStarBuilder.setUnparsedTarget(name) case _ => - builder.getUnresolvedAttributeBuilder.setUnparsedIdentifier(name) + val attributeBuilder = builder.getUnresolvedAttributeBuilder.setUnparsedIdentifier(name) + planId.foreach(attributeBuilder.setPlanId) } } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 560276d154e4..87aadfe437be 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -117,6 +117,8 @@ import org.apache.spark.util.Utils */ class Dataset[T] private[sql] (val session: SparkSession, private[sql] val plan: proto.Plan) extends Serializable { + // Make sure we don't forget to set plan id. + assert(plan.getRoot.getCommon.hasPlanId) override def toString: String = { try { @@ -873,7 +875,14 @@ class Dataset[T] private[sql] (val session: SparkSession, private[sql] val plan: * @group untypedrel * @since 3.4.0 */ - def col(colName: String): Column = functions.col(colName) + def col(colName: String): Column = { + val planId = if (plan.getRoot.hasCommon && plan.getRoot.getCommon.hasPlanId) { + Option(plan.getRoot.getCommon.getPlanId) + } else { + None + } + Column.apply(colName, planId) + } /** * Selects column based on the column name specified as a regex and returns it as [[Column]]. diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 0e5aaace20d6..a63c23e6bf1e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql import java.io.Closeable +import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ @@ -48,7 +49,10 @@ import org.apache.spark.sql.connect.client.util.Cleaner * .getOrCreate() * }}} */ -class SparkSession(private val client: SparkConnectClient, private val cleaner: Cleaner) +class SparkSession( + private val client: SparkConnectClient, + private val cleaner: Cleaner, + private val planIdGenerator: AtomicLong) extends Serializable with Closeable with Logging { @@ -183,6 +187,7 @@ class SparkSession(private val client: SparkConnectClient, private val cleaner: private[sql] def newDataset[T](f: proto.Relation.Builder => Unit): Dataset[T] = { val builder = proto.Relation.newBuilder() f(builder) + builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) val plan = proto.Plan.newBuilder().setRoot(builder).build() new Dataset[T](this, plan) } @@ -204,6 +209,15 @@ class SparkSession(private val client: SparkConnectClient, private val cleaner: client.execute(plan).asScala.foreach(_ => ()) } + /** + * This resets the plan id generator so we can produce plans that are comparable. + * + * For testing only! + */ + private[sql] def resetPlanIdGenerator(): Unit = { + planIdGenerator.set(0) + } + override def close(): Unit = { client.shutdown() allocator.close() @@ -213,9 +227,11 @@ class SparkSession(private val client: SparkConnectClient, private val cleaner: // The minimal builder needed to create a spark session. // TODO: implements all methods mentioned in the scaladoc of [[SparkSession]] object SparkSession extends Logging { + private val planIdGenerator = new AtomicLong + def builder(): Builder = new Builder() - private lazy val cleaner = { + private[sql] lazy val cleaner = { val cleaner = new Cleaner cleaner.start() cleaner @@ -238,7 +254,7 @@ object SparkSession extends Logging { if (_client == null) { _client = SparkConnectClient.builder().build() } - new SparkSession(_client, cleaner) + new SparkSession(_client, cleaner, planIdGenerator) } } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index c9d9f43cbaff..7077d8c94e8d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -27,7 +27,7 @@ import org.apache.commons.io.output.TeeOutputStream import org.scalactic.TolerantNumerics import org.apache.spark.sql.connect.client.util.{IntegrationTestUtils, RemoteSparkSession} -import org.apache.spark.sql.functions.{aggregate, array, col, lit, sequence, shuffle, transform, udf} +import org.apache.spark.sql.functions.{aggregate, array, col, lit, rand, sequence, shuffle, transform, udf} import org.apache.spark.sql.types._ class ClientE2ETestSuite extends RemoteSparkSession { @@ -399,4 +399,11 @@ class ClientE2ETestSuite extends RemoteSparkSession { .getSeq[Int](0) assert(result.toSet === Set(1, 2, 3, 74)) } + + test("ambiguous joins") { + val left = spark.range(100).select(col("id"), rand(10).as("a")) + val right = spark.range(100).select(col("id"), rand(12).as("a")) + val joined = left.join(right, left("id") === right("id")).select(left("id"), right("a")) + assert(joined.schema.catalogString === "struct") + } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 412371c41863..c3c80a08379f 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -16,11 +16,11 @@ */ package org.apache.spark.sql -import scala.collection.JavaConverters._ +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicLong import io.grpc.Server -import io.grpc.netty.NettyServerBuilder -import java.util.concurrent.TimeUnit +import io.grpc.inprocess.{InProcessChannelBuilder, InProcessServerBuilder} import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite @@ -40,33 +40,27 @@ class DatasetSuite private var service: DummySparkConnectService = _ private var ss: SparkSession = _ - private def getNewSparkSession(port: Int): SparkSession = { - assert(port != 0) - SparkSession - .builder() - .client( - SparkConnectClient - .builder() - .connectionString(s"sc://localhost:$port") - .build()) - .build() + private def newSparkSession(): SparkSession = { + val client = new SparkConnectClient( + proto.UserContext.newBuilder().build(), + InProcessChannelBuilder.forName(getClass.getName).directExecutor().build(), + "test") + new SparkSession(client, cleaner = SparkSession.cleaner, planIdGenerator = new AtomicLong) } private def startDummyServer(): Unit = { service = new DummySparkConnectService() - val sb = NettyServerBuilder - // Let server bind to any free port - .forPort(0) + server = InProcessServerBuilder + .forName(getClass.getName) .addService(service) - - server = sb.build + .build() server.start() } override def beforeEach(): Unit = { super.beforeEach() startDummyServer() - ss = getNewSparkSession(server.getPort) + ss = newSparkSession() } override def afterEach(): Unit = { @@ -76,47 +70,6 @@ class DatasetSuite } } - test("limit") { - val df = ss.newDataset(_ => ()) - val builder = proto.Relation.newBuilder() - builder.getLimitBuilder.setInput(df.plan.getRoot).setLimit(10) - - val expectedPlan = proto.Plan.newBuilder().setRoot(builder).build() - df.limit(10).analyze - val actualPlan = service.getAndClearLatestInputPlan() - assert(actualPlan.equals(expectedPlan)) - } - - test("select") { - val df = ss.newDataset(_ => ()) - - val builder = proto.Relation.newBuilder() - val dummyCols = Seq[Column](Column("a"), Column("b")) - builder.getProjectBuilder - .setInput(df.plan.getRoot) - .addAllExpressions(dummyCols.map(_.expr).asJava) - val expectedPlan = proto.Plan.newBuilder().setRoot(builder).build() - - df.select(dummyCols: _*).analyze - val actualPlan = service.getAndClearLatestInputPlan() - assert(actualPlan.equals(expectedPlan)) - } - - test("filter") { - val df = ss.newDataset(_ => ()) - - val builder = proto.Relation.newBuilder() - val dummyCondition = Column.fn("dummy func", Column("a")) - builder.getFilterBuilder - .setInput(df.plan.getRoot) - .setCondition(dummyCondition.expr) - val expectedPlan = proto.Plan.newBuilder().setRoot(builder).build() - - df.filter(dummyCondition).analyze - val actualPlan = service.getAndClearLatestInputPlan() - assert(actualPlan.equals(expectedPlan)) - } - test("write") { val df = ss.newDataset(_ => ()).limit(10) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala index b759471e777f..f7589d957ca5 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.sql import java.nio.file.{Files, Path} import java.util.Collections +import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable import scala.util.{Failure, Success, Try} import com.google.protobuf.util.JsonFormat import io.grpc.inprocess.InProcessChannelBuilder -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.funsuite.{AnyFunSuite => ConnectFunSuite} // scalastyle:ignore funsuite import org.apache.spark.connect.proto @@ -55,7 +56,11 @@ import org.apache.spark.sql.types._ * `connector/connect/server` module */ // scalastyle:on -class PlanGenerationTestSuite extends ConnectFunSuite with BeforeAndAfterAll with Logging { +class PlanGenerationTestSuite + extends ConnectFunSuite + with BeforeAndAfterAll + with BeforeAndAfterEach + with Logging { // Borrowed from SparkFunSuite private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" @@ -102,8 +107,12 @@ class PlanGenerationTestSuite extends ConnectFunSuite with BeforeAndAfterAll wit val client = SparkConnectClient( proto.UserContext.newBuilder().build(), InProcessChannelBuilder.forName("/dev/null").build()) - val builder = SparkSession.builder().client(client) - session = builder.build() + session = + new SparkSession(client, cleaner = SparkSession.cleaner, planIdGenerator = new AtomicLong) + } + + override protected def beforeEach(): Unit = { + session.resetPlanIdGenerator() } override protected def afterAll(): Unit = { @@ -361,7 +370,8 @@ class PlanGenerationTestSuite extends ConnectFunSuite with BeforeAndAfterAll wit } test("apply") { - simple.select(simple.apply("a")) + val stable = simple + stable.select(stable("a")) } test("hint") { @@ -369,7 +379,8 @@ class PlanGenerationTestSuite extends ConnectFunSuite with BeforeAndAfterAll wit } test("col") { - simple.select(simple.col("id"), simple.col("b")) + val stable = simple + stable.select(stable.col("id"), stable.col("b")) } test("colRegex") { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala deleted file mode 100644 index 8410073d6ec5..000000000000 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql - -import io.grpc.Server -import io.grpc.netty.NettyServerBuilder -import java.util.concurrent.TimeUnit -import org.scalatest.BeforeAndAfterEach -import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite - -import org.apache.spark.connect.proto -import org.apache.spark.sql.connect.client.{DummySparkConnectService, SparkConnectClient} - -class SparkSessionSuite - extends AnyFunSuite // scalastyle:ignore funsuite - with BeforeAndAfterEach { - - private var server: Server = _ - private var service: DummySparkConnectService = _ - private val SERVER_PORT = 15250 - - private def startDummyServer(port: Int): Unit = { - service = new DummySparkConnectService() - val sb = NettyServerBuilder - .forPort(port) - .addService(service) - - server = sb.build - server.start() - } - - override def beforeEach(): Unit = { - super.beforeEach() - startDummyServer(SERVER_PORT) - } - - override def afterEach(): Unit = { - if (server != null) { - server.shutdownNow() - assert(server.awaitTermination(5, TimeUnit.SECONDS), "server failed to shutdown") - } - } - - test("SparkSession initialisation with connection string") { - val ss = SparkSession - .builder() - .client( - SparkConnectClient - .builder() - .connectionString(s"sc://localhost:$SERVER_PORT") - .build()) - .build() - val plan = proto.Plan.newBuilder().build() - ss.analyze(plan, proto.Explain.ExplainMode.SIMPLE) - assert(plan.equals(service.getAndClearLatestInputPlan())) - } - - private def rangePlanCreator( - start: Long, - end: Long, - step: Long, - numPartitions: Option[Int]): proto.Plan = { - val builder = proto.Relation.newBuilder() - val rangeBuilder = builder.getRangeBuilder - .setStart(start) - .setEnd(end) - .setStep(step) - numPartitions.foreach(rangeBuilder.setNumPartitions) - proto.Plan.newBuilder().setRoot(builder).build() - } - - private def testRange( - start: Long, - end: Long, - step: Long, - numPartitions: Option[Int], - failureHint: String): Unit = { - val expectedPlan = rangePlanCreator(start, end, step, numPartitions) - val actualPlan = service.getAndClearLatestInputPlan() - assert(actualPlan.equals(expectedPlan), failureHint) - } - - test("range query") { - val ss = SparkSession - .builder() - .client( - SparkConnectClient - .builder() - .connectionString(s"sc://localhost:$SERVER_PORT") - .build()) - .build() - - ss.range(10).analyze - testRange(0, 10, 1, None, "Case: range(10)") - - ss.range(0, 20).analyze - testRange(0, 20, 1, None, "Case: range(0, 20)") - - ss.range(6, 20, 3).analyze - testRange(6, 20, 3, None, "Case: range(6, 20, 3)") - - ss.range(10, 100, 5, 2).analyze - testRange(10, 100, 5, Some(2), "Case: range(6, 20, 3, Some(2))") - } - -} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 98dacbcab891..8e827007950b 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite import org.apache.spark.connect.proto import org.apache.spark.connect.proto.{AnalyzePlanRequest, AnalyzePlanResponse, ExecutePlanRequest, ExecutePlanResponse, SparkConnectServiceGrpc} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connect.common.config.ConnectCommon class SparkConnectClientSuite @@ -33,20 +34,23 @@ class SparkConnectClientSuite with BeforeAndAfterEach { private var client: SparkConnectClient = _ + private var service: DummySparkConnectService = _ private var server: Server = _ private def startDummyServer(port: Int): Unit = { - val sb = NettyServerBuilder + service = new DummySparkConnectService + server = NettyServerBuilder .forPort(port) - .addService(new DummySparkConnectService()) - - server = sb.build + .addService(service) + .build() server.start() } + override def beforeEach(): Unit = { super.beforeEach() client = null server = null + service = null } override def afterEach(): Unit = { @@ -104,6 +108,16 @@ class SparkConnectClientSuite } } + test("SparkSession initialisation with connection string") { + val testPort = 16002 + client = SparkConnectClient.builder().connectionString(s"sc://localhost:$testPort").build() + startDummyServer(testPort) + val session = SparkSession.builder().client(client).build() + val df = session.range(10) + df.analyze // Trigger RPC + assert(df.plan === service.getAndClearLatestInputPlan()) + } + private case class TestPackURI( connectionString: String, isCorrect: Boolean, diff --git a/connector/connect/common/src/test/resources/query-tests/queries/alias_string.json b/connector/connect/common/src/test/resources/query-tests/queries/alias_string.json index 1209a4dfb877..98ea62f986b9 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/alias_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/alias_string.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "subqueryAlias": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/alias_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/alias_string.proto.bin index 1f969506e38e..6e8467cccde1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/alias_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/alias_string.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.json b/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.json index ba861c077fbc..b469cbd0a335 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "subqueryAlias": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.proto.bin index 788038972d89..7034d39cd8a5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/alias_symbol.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/apply.json b/connector/connect/common/src/test/resources/query-tests/queries/apply.json index d53d24efef80..e484781708eb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/apply.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/apply.json @@ -1,13 +1,20 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "expressions": [{ "unresolvedAttribute": { - "unparsedIdentifier": "a" + "unparsedIdentifier": "a", + "planId": "0" } }] } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/apply.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/apply.proto.bin index 80cb4ad244ee..5d5efcead5e1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/apply.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/apply.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/as_string.json b/connector/connect/common/src/test/resources/query-tests/queries/as_string.json index 501189fbe13a..d74c9d16a7ff 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/as_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/as_string.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "subqueryAlias": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/as_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/as_string.proto.bin index f433909e126a..829d6083e094 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/as_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/as_string.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.json b/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.json index e2a7d2bb193c..ca69a743175f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "subqueryAlias": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.proto.bin index c9ad3f8e0c5a..f7111a4651d9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/as_symbol.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/coalesce.json b/connector/connect/common/src/test/resources/query-tests/queries/coalesce.json index 34a329f893ae..cb08412296aa 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/coalesce.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/coalesce.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "repartition": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/coalesce.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/coalesce.proto.bin index ed1a6ce29fab..b03e7d58a2bf 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/coalesce.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/coalesce.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/col.json b/connector/connect/common/src/test/resources/query-tests/queries/col.json index aa2c09ce0d75..f3abc8a81aff 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/col.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/col.json @@ -1,17 +1,25 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "expressions": [{ "unresolvedAttribute": { - "unparsedIdentifier": "id" + "unparsedIdentifier": "id", + "planId": "0" } }, { "unresolvedAttribute": { - "unparsedIdentifier": "b" + "unparsedIdentifier": "b", + "planId": "0" } }] } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/col.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/col.proto.bin index 6db0c6bb0b05..15c4eabb8d50 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/col.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/col.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/colRegex.json b/connector/connect/common/src/test/resources/query-tests/queries/colRegex.json index 90d35213bbbf..56021594c881 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/colRegex.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/colRegex.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "2" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/colRegex.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/colRegex.proto.bin index 6503926be7a4..2f3ab10233ef 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/colRegex.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/colRegex.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json b/connector/connect/common/src/test/resources/query-tests/queries/column_add.json index c7c88bdc5d11..cfa40fac8c6f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_add.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_add.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin index 26ccd472276e..10b410b5b08b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json index 559d7c7b254d..4fe650db9d3b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin index b05b217b0617..e9b907683287 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_alias.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_and.json b/connector/connect/common/src/test/resources/query-tests/queries/column_and.json index 1faa3965b12a..d3f8cd0e73cb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_and.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_and.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin index a566569f2930..241f1a9303b2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_apply.json b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.json index c72cdda0c299..b203a20a0ea6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_apply.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_apply.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.proto.bin index bbc3bacbd9cc..9e56d5891f50 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_apply.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_apply.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.json b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.json index 15c6c6cf0935..426fd1fbb759 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin index 4762a9fad8c4..602beafb01cb 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.json b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.json index dff2912f8b6c..e943c01f26fb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin index 83592a71e5ca..2952e871f6e6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc.json b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.json index 11d2915091c3..31f3102f77a4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_asc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.proto.bin index 56e5c5d20763..ee5bda529c45 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_asc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_asc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json index 11d2915091c3..31f3102f77a4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin index 56e5c5d20763..ee5bda529c45 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json index eac9be9e5ac1..94326e0f6621 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin index 21665330de2e..496fe40192da 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_between.json b/connector/connect/common/src/test/resources/query-tests/queries/column_between.json index 8473a3e7cdaa..20927b93d843 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_between.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_between.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin index c651f8856bbb..d03dd02a2f36 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json index 17431e74d942..bd3ac671fca3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin index 3517777d8afa..4815bc7dd1a2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.json b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.json index cfd8e2dfb7f1..eaa27ffa4616 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin index 16ea227007bb..9cf110da4ad6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json index d3e4c326c26c..c51eb3140c33 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin index 2c154418bb7e..70c61f962057 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_cast.json b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.json index d014ce98803e..1a1ee5ed4d51 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_cast.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_cast.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.proto.bin index 3b3d04193ac0..60e807b4c350 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_cast.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_cast.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_contains.json b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.json index 862dd8cefa00..05d6ccf38b36 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_contains.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_contains.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.proto.bin index 12a45de9ce60..9c796f9470c3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_contains.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_contains.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc.json b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.json index f882e4abad25..50efda387ec4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_desc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.proto.bin index 3d9b64634d9d..df2589d8231b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_desc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_desc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json index 707ced41a162..bed300feea2e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin index 54ae15ef70be..b8caacc55b9e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json index f882e4abad25..50efda387ec4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin index 3d9b64634d9d..df2589d8231b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_divide.json b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.json index 081717618223..8d71061b151c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_divide.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_divide.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.proto.bin index 856a75655915..49b5d8d2590d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_divide.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_divide.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.json b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.json index ff39dfa6532a..92639eeedc67 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin index b95d8d3dcd60..edafc8b1f1f5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.json b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.json index 99edefdc9a43..f4171c2792fb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin index c854a66c726c..03f41a339f00 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json index 3481ee46e5a5..eea1da49bc59 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin index d38e1836b3fe..22de941ad44b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json index 834afc4dee7a..7397f4fb46ac 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin index e8505e38fdca..e226de59ddcd 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_geq.json b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.json index aca63fadf09c..9f24bc251739 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_geq.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin index c0762cef053d..1c4af866109a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getField.json b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.json index d68d274a406d..21d5bb6f23d8 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_getField.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getField.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.proto.bin index eac146d122fb..c76b69bf5fa4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_getField.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_getField.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.json b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.json index 41eb3eadafcb..e3bfd3d6e842 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.proto.bin index 92083231b141..9120801100fe 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_getItem.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_gt.json b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.json index 658a17c00ab3..4bb8fb41f249 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_gt.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin index 9304b3dae72c..44ca37fbb404 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.json b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.json index bc751f0ce1d5..47c1b63abe31 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin index 10226e50caa2..285400db7daf 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.json index 0eb722a08ecc..f594918ed930 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin index 1f96b82d070b..1030abda5b8c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json index bb2403f1d193..f34d3f4eac55 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin index 45574baf87cc..e8cccdf02493 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.json index 386ad8898eb0..74e990622a3a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin index 44b48593580a..8fc24a9e21b3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isin.json b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.json index 1a990cc08516..d8811a4e780b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_isin.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin index 9e61c9d26d6b..365e07f35bb4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_leq.json b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.json index 9ea4d99f9b7c..cda8694c0439 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_leq.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin index 7f62d95ee34b..e8463292e404 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_like.json b/connector/connect/common/src/test/resources/query-tests/queries/column_like.json index 2afab4b03297..1390451af55a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_like.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_like.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_like.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_like.proto.bin index dee7e79301ce..07382ec1643c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_like.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_like.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_lt.json b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.json index 019e33cd1064..c927e75de181 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_lt.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin index ff658957ceef..f4c3a110b126 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.json b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.json index 7cbe8a11fd9d..0c5a78eea2df 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin index 461b09b0b825..55bfeba04ed6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.json b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.json index 58ebb2604dac..8c17581c67d1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.proto.bin index 4ca350de10b1..8fd1b3941d1f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_multiply.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not.json b/connector/connect/common/src/test/resources/query-tests/queries/column_not.json index 39417760083b..2f873196ba1d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_not.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_not.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin index 59a14a401985..19609b6ee85a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.json b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.json index de8f07fb4786..589d57a18b94 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin index 006450915a3e..cdf0b4290e61 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_or.json b/connector/connect/common/src/test/resources/query-tests/queries/column_or.json index 091280b2ae6f..ae1424f763fe 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_or.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_or.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin index aaf78e293e9a..69f219e938a4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.json b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.json index 490766a88d79..e53403db41cd 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.proto.bin index 7e2b6996c70d..7dd56baf0421 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_rlike.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star.json b/connector/connect/common/src/test/resources/query-tests/queries/column_star.json index ecb84fee1745..ef88067a7a4c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_star.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_star.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_star.proto.bin index bde97f5f8ef6..3ca04d082d76 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_star.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_star.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.json b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.json index 26372cb4e501..e1159b5673db 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin index 63c5c7f5330d..af744b7d6b47 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.json b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.json index fe66f718e23c..431e13d81863 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin index dd10f51fcdd2..fa1132c73de7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_substr.json b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.json index 6c074decbba6..3b02117cc6e5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_substr.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin index e050495220d5..636a46a48062 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.json b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.json index 79a42858f09c..d15c2941ee1b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin index 794972c95bf7..f5716427588e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json index 5e39ebe6074b..0db558e49e38 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin index 7ff0a7c8ad51..66343bea4e29 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json index 8c0004a6abb4..db2ceccfd22a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin index 51e8c7ce37e1..031c3683c5e6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_withField.json b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.json index ba9d22bcc5d1..86b9396a4e13 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/column_withField.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/column_withField.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.proto.bin index 33be23b4c61c..a413740fa505 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/column_withField.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/column_withField.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.json b/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.json index bac0eebf8ee8..fecba18096bb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.proto.bin index b86654bb57fe..ff6d2f3b4a7a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/crossJoin.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/cube_column.json b/connector/connect/common/src/test/resources/query-tests/queries/cube_column.json index 49016593a34c..5b9709ff0657 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/cube_column.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/cube_column.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/cube_column.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/cube_column.proto.bin index c706144de595..d46e40b39dcf 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/cube_column.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/cube_column.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/cube_string.json b/connector/connect/common/src/test/resources/query-tests/queries/cube_string.json index 49016593a34c..5b9709ff0657 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/cube_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/cube_string.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/cube_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/cube_string.proto.bin index c706144de595..d46e40b39dcf 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/cube_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/cube_string.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/describe.json b/connector/connect/common/src/test/resources/query-tests/queries/describe.json index c3e9ded31334..d767db5241f4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/describe.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/describe.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "describe": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/describe.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/describe.proto.bin index a39ee7fc80ce..8a2117e519f6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/describe.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/describe.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/distinct.json b/connector/connect/common/src/test/resources/query-tests/queries/distinct.json index 094caa142080..ae796b520353 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/distinct.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/distinct.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "deduplicate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/distinct.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/distinct.proto.bin index 4892a7c71625..07430c438310 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/distinct.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/distinct.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.json b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.json index 094caa142080..ae796b520353 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "deduplicate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.proto.bin index 4892a7c71625..07430c438310 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.json b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.json index e55248a383ef..e72e23c86caf 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "deduplicate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.proto.bin index 51d419de5709..c8e3885fbf80 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.json b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.json index d2a9e0414187..754cecac4b25 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "deduplicate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.proto.bin index 57c97010bddb..1a2d635e58e5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.json b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.json index 7b2468c70a69..c4a8df30c586 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "deduplicate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.proto.bin index 496dc9e45164..719a373c2e38 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.json b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.json index 275f15a04e16..6a8546d9326c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "drop": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.proto.bin index 65d2cac04cc6..f4585af804ae 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_column.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.json index 96d50cef533a..69c31ec58585 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "drop": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.proto.bin index 29ae3de0fcca..c085f69fc544 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/drop_multiple_strings.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.json b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.json index 454bb1744b85..7f4cd227186e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "drop": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.proto.bin index 3cf2a6b652ad..37d71479cdb8 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_column.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.json b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.json index 812fc60d9d2f..7e4c4e8feb1b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "drop": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.proto.bin index a4146717a0bf..9d704de48c88 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/drop_single_string.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/except.json b/connector/connect/common/src/test/resources/query-tests/queries/except.json index d88134dc2689..6544e03f6e10 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/except.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/except.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "setOp": { "leftInput": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "rightInput": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/except.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/except.proto.bin index 64fcb2a45c9d..0e9efea2f94d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/except.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/except.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.json b/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.json index dfb04bf53a18..e77b583b9c28 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "setOp": { "leftInput": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "rightInput": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.proto.bin index cc746e54e002..19f9231eb267 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/exceptAll.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/filter.json b/connector/connect/common/src/test/resources/query-tests/queries/filter.json index eeb5c2695418..1046e1262150 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/filter.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/filter.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "filter": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/filter.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/filter.proto.bin index adaec008b76e..069171ead323 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/filter.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/filter.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.json b/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.json index 77635b511722..a2c49ec98c61 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "filter": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.proto.bin index e7ddb9c54fa0..56e5be565435 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/filter_expr.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_abs.json b/connector/connect/common/src/test/resources/query-tests/queries/function_abs.json index 16002c3f4d10..13df3437ddab 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_abs.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_abs.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_abs.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_abs.proto.bin index 3f56daaa382c..86cfbc09a8f9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_abs.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_abs.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_acos.json b/connector/connect/common/src/test/resources/query-tests/queries/function_acos.json index 01f3eb936eba..7506c0f6cb63 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_acos.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_acos.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_acos.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_acos.proto.bin index e42dfe0c28d9..cc6a279cb188 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_acos.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_acos.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.json b/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.json index d3af379493c3..6a83b4ab008b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.proto.bin index 5d6e171aee59..e16ed2ba92e3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_acosh.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.json b/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.json index 2a9a5f176401..b1b2e78a0843 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.proto.bin index b257a77e26b6..6abacc9cc2b4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_add_months.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.json b/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.json index 833350ad0b39..3116837aeb87 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.proto.bin index 5fec870563f7..f97843e086a5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_aggregate.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json index 069fe54c6dc6..5579faf11964 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.proto.bin index 1c06f9004475..bac82f670b29 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json index fbdc69cd793c..851862082ca0 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.proto.bin index 12f2ec5362d8..fd61420fd1e4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array.json index 7c75a9def64e..20fe495bb9bf 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array.proto.bin index 0a5b09e8b995..2b679eb4c6db 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.json index 7e2a7e300a28..cabd44c063de 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.proto.bin index 9832323a527b..76f2f0255bf2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_append.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.json index 4c81b0c8b388..c3ebf313190c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.proto.bin index 28a30d579b25..949d66cb951f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_compact.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.json index 6e43c2768db7..a362d66d9d64 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.proto.bin index cdb20044cc99..d8764f60364c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_contains.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json index c512dd362c46..d38f4194bcd2 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.proto.bin index 5adda1b1f453..e6359c074bf2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_distinct.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.json index f27cce411990..17d50c87161d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.proto.bin index a7a06cb7ec3e..692511b2f74a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_except.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.json index 6c6b95933933..f4540edbf410 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.proto.bin index c18114edbd01..6e2178ad124e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_insert.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json index 5149f675e8cb..1b95a6724f86 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.proto.bin index bab4ead039ae..67fb497cf270 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_intersect.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.json index 8dcaad72daf4..94e8c176cefb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.proto.bin index db6dc1adff84..fbab1b208605 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.json index bbad2109ff27..ad580c33e476 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.proto.bin index 0604d19d0c92..e3fb6b3bf67c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.json index 1270be928a92..ba67984758a5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.proto.bin index f96db47323ae..f7a98c08cd17 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_max.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.json index e36f2275c4f0..a342ae18f9ef 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.proto.bin index 4b01ad396748..02cfdfeb215d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_min.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.json index d13d31d467b8..4c212cb02827 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.proto.bin index c26fbff6f069..4ef2b11273f2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_position.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.json index 99fa5f3a1bb7..8c562247714a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.proto.bin index 68b37cdbfbe9..95e2872ad77b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_remove.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json index b445ce38ee03..c9d9f1f9ca79 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.proto.bin index 186b8027beac..e370db16e977 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_repeat.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.json index 2e851777cbff..406dc54c8cd2 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.proto.bin index da38e74e5db2..2074caae1638 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json index df8c94c62573..95be74d0b4c8 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.proto.bin index 3217e0b59d17..c1e2363f0fda 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.json b/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.json index f6e83f316c73..7d54079cdb47 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.proto.bin index 951d56813992..fc3d9d7cd0fd 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_array_union.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json index 94c87c477dbf..ce1d288e00d7 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.proto.bin index 771dd700f13f..216f306507d4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json index 80d9c8c85c2c..14769082725f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.proto.bin index ea88aa580414..609f52db3247 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asc.json b/connector/connect/common/src/test/resources/query-tests/queries/function_asc.json index 9d963d55ec18..30740c81ba41 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_asc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_asc.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_asc.proto.bin index 8236485b4f61..7c5bc4213a6f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_asc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_asc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.json b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.json index 9d963d55ec18..30740c81ba41 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.proto.bin index 8236485b4f61..7c5bc4213a6f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_first.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.json b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.json index cf6adf0cc364..b8bbbb73544f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.proto.bin index 8292c9d24d71..1eb6f88cac87 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_asc_nulls_last.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.json b/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.json index 5d815ff83bfe..3c4dcb70fead 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.proto.bin index 26505e959121..5989bd3b5c60 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_ascii.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asin.json b/connector/connect/common/src/test/resources/query-tests/queries/function_asin.json index fd61c2ffbf1c..4bf89be75345 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_asin.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_asin.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asin.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_asin.proto.bin index 3b4b86cc15b9..737ad789da26 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_asin.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_asin.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.json b/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.json index 6d41470ca615..238571b0231c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.proto.bin index c125ca3114aa..01ea4675b22e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_asinh.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json b/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json index 35312d1fbb3c..5520b70a0250 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.proto.bin index b88dab9c58ed..6992604efe1b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_atan.json b/connector/connect/common/src/test/resources/query-tests/queries/function_atan.json index bab8ef27ec3f..3ae4e7ef188e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_atan.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_atan.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_atan.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_atan.proto.bin index e946720aad4c..b932086941f4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_atan.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_atan.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.json b/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.json index 04a3490df12d..7d08116c40ae 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.proto.bin index 3271fecbf812..372ae8358494 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_atan2.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.json b/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.json index 2ae1d25ed06e..8daec8813917 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.proto.bin index 89c594f4baa3..0aa2f3527ae9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_atanh.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_avg.json b/connector/connect/common/src/test/resources/query-tests/queries/function_avg.json index f9e582b9219f..b433f1ea89c2 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_avg.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_avg.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_avg.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_avg.proto.bin index 24bbef4be5e3..9d9bd296dbdd 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_avg.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_avg.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_base64.json b/connector/connect/common/src/test/resources/query-tests/queries/function_base64.json index fc8e388be9ba..97739dca283e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_base64.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_base64.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_base64.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_base64.proto.bin index bcc329dc78b4..fc854d974752 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_base64.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_base64.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bin.json b/connector/connect/common/src/test/resources/query-tests/queries/function_bin.json index 658990a6f907..304e56504bad 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_bin.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_bin.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bin.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_bin.proto.bin index d499f54e7217..e8d55fb8d614 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_bin.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_bin.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.json b/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.json index 771ac0fdc021..df21871cb535 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.proto.bin index 52c0ca8df49e..860c2eaec0e8 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_bit_length.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.json b/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.json index c350c65312dd..7ddf73253e0a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.proto.bin index 9acca430f5f7..bfaefb2a2007 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_bitwise_not.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bround.json b/connector/connect/common/src/test/resources/query-tests/queries/function_bround.json index c90800492124..585a0befb224 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_bround.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_bround.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bround.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_bround.proto.bin index 90b949a0a680..8625ccb1a58f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_bround.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_bround.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.json b/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.json index 4ec5fb5f27b4..971660144a5b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.proto.bin index 4ccecb3d59c2..1b389401f15e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_bucket.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.json b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.json index 5f3753452ea8..5a9961ab47f5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.proto.bin index ce56aea7a569..3761deb1663a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json index 94a61a5fd05f..bda5e85924c3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.proto.bin index 4d3b25dada47..8db402ac167e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.json b/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.json index aa36f12c0540..497922b5df75 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.proto.bin index 36b892c64763..ec871018489c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_coalesce.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_col.json b/connector/connect/common/src/test/resources/query-tests/queries/function_col.json index 63988117f431..0420a3d12f6f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_col.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_col.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_col.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_col.proto.bin index 4a4eb9b59869..e113880f31b7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_col.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_col.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.json b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.json index 109392a141e8..c5bae4baef35 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.proto.bin index b96919003c0d..e3827b9f650a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_list.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.json b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.json index fa06e613e75b..615386d050e1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.proto.bin index dda4b4f0ff25..5fb97f27d25b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_collect_set.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_concat.json b/connector/connect/common/src/test/resources/query-tests/queries/function_concat.json index c25176929770..4a053d9c3c35 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_concat.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_concat.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_concat.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_concat.proto.bin index 293ab94e506d..e53eb7a75b8a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_concat.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_concat.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json b/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json index bd88b533d6a9..b9ba89b42185 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.proto.bin index 7da2b0704931..2fbc4f709044 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_concat_ws.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_conv.json b/connector/connect/common/src/test/resources/query-tests/queries/function_conv.json index 6ea114c4511d..c6734936bfcd 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_conv.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_conv.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_conv.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_conv.proto.bin index 9dd42bd34c0d..373b997b7924 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_conv.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_conv.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_corr.json b/connector/connect/common/src/test/resources/query-tests/queries/function_corr.json index d1fc5417f8d2..6fadb0385622 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_corr.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_corr.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_corr.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_corr.proto.bin index d8bb18b3c064..fdeeb4fd12d1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_corr.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_corr.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cos.json b/connector/connect/common/src/test/resources/query-tests/queries/function_cos.json index f53521db9c28..f7072dff0340 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_cos.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_cos.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cos.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_cos.proto.bin index d21e15b0aa08..09fd198b097c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_cos.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_cos.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.json b/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.json index 151c7f835eb8..3bcab61d37a0 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.proto.bin index 202a4ba8a8fe..54d5da8fabfa 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_cosh.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cot.json b/connector/connect/common/src/test/resources/query-tests/queries/function_cot.json index 2100af7d09ea..62ce963fa873 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_cot.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_cot.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cot.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_cot.proto.bin index d26a2d30cca6..e79c32660a77 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_cot.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_cot.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_count.json b/connector/connect/common/src/test/resources/query-tests/queries/function_count.json index 77eff56f6ef4..126a0ca242c5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_count.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_count.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_count.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_count.proto.bin index 97667ca83602..6c87a809ad0c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_count.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_count.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json b/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json index d33f9d958e17..eb211ceb239a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.proto.bin index a4582f980132..591e2300ec68 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_countDistinct.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json index a1227d5b54a4..3c4df70a5fbf 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.proto.bin index ffcfae52acb8..4a7202f15e76 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_pop.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json index c7d6ddf1273e..7c723069e467 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.proto.bin index c830d80acc3e..ebff687730e3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_covar_samp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.json b/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.json index ecb8f964a848..1892a9af85d9 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.proto.bin index ceb4e60fa912..54ad14dedae4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_crc32.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_csc.json b/connector/connect/common/src/test/resources/query-tests/queries/function_csc.json index ceb722b1b1f1..88504ed9c528 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_csc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_csc.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_csc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_csc.proto.bin index 147af78100dd..0ed5022a73ad 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_csc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_csc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json b/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json index 4e22d94aa305..ac4884119907 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.proto.bin index fe9c87a0a03e..7578245aabe3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_cume_dist.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.json b/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.json index 0c4d080f71dc..6dab8c39d626 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.proto.bin index ef6fcd56b179..f32c3f541c4c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_current_date.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json index 48e5b84689c2..16af5eb9ba08 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.proto.bin index b4f0e65d307c..5a1f3de6c3a9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.json b/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.json index 6453851f7bcd..f81ad3335242 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.proto.bin index ce4b1a6e8bb8..f4dbc16b05c1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_date_add.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.json b/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.json index 8a65dbcb016e..9b3d469ed4e9 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.proto.bin index f1322a228e99..7226c20974b2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_date_format.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.json b/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.json index 90d9ea0f7adf..f1dde0902a20 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.proto.bin index fe779f24a10b..43b630c27ed4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_date_sub.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.json b/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.json index a1cadc2b2699..363da9b9b900 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.proto.bin index 7fcaf3be6bd4..f037fb8d34a5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_date_trunc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.json b/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.json index 4ab99ffbbed0..b5ef560486d0 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.proto.bin index 0f567ba2a940..02e917b40683 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_datediff.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json index 4e9156323cb5..3e453c1f7a65 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.proto.bin index d0a12815c273..3a2973e21e5a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json index b8ff1c59613a..74715de151e7 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.proto.bin index 9a4d4cc7735d..fceea203c790 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofweek.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json index 047142c10b0e..d23c6790a47d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.proto.bin index 5d8efe15f668..a526b449ae0a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_dayofyear.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_days.json b/connector/connect/common/src/test/resources/query-tests/queries/function_days.json index 8fb62b4a2e4b..9e20c48729a3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_days.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_days.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_days.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_days.proto.bin index ecfa97f445c9..b0a8472f8c4f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_days.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_days.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_decode.json b/connector/connect/common/src/test/resources/query-tests/queries/function_decode.json index 67ec30bf4c3c..6be60808e64f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_decode.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_decode.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_decode.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_decode.proto.bin index 952b5c6ed04c..18b8bbcf6a01 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_decode.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_decode.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.json b/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.json index 152c92cbe6cb..e096b07e4dc6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.proto.bin index 772d2ee9d3b1..e2d264bb2e10 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_degrees.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.json b/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.json index 3cc81b32613f..46c5e1eaddfc 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.proto.bin index 2df47f4ce657..4597e63be837 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_dense_rank.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_desc.json b/connector/connect/common/src/test/resources/query-tests/queries/function_desc.json index 8f1d61d70ac8..0841b33b8fb6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_desc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_desc.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_desc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_desc.proto.bin index a1237f80486f..bd549431a832 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_desc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_desc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.json b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.json index fa0654bbda7d..683de2af2388 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.proto.bin index 22de37fb20f4..b46e09d6ef3d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_first.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.json b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.json index 8f1d61d70ac8..0841b33b8fb6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.proto.bin index a1237f80486f..bd549431a832 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_desc_nulls_last.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.json b/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.json index a037f4155e2a..ef5551440934 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.proto.bin index ceb7df30178c..993818c6cb4b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_element_at.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_encode.json b/connector/connect/common/src/test/resources/query-tests/queries/function_encode.json index 1f3767de7c85..92e95f2c946d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_encode.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_encode.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_encode.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_encode.proto.bin index e46a529ccf4d..9644825af470 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_encode.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_encode.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_exists.json b/connector/connect/common/src/test/resources/query-tests/queries/function_exists.json index abf4410f76ff..76d107092ae1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_exists.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_exists.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_exists.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_exists.proto.bin index edd91a6c3415..27fbc03c6988 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_exists.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_exists.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_exp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_exp.json index a563d4dfbe52..d317efef75ee 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_exp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_exp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_exp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_exp.proto.bin index 290c68a7f1f8..7def20c94df0 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_exp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_exp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_explode.json b/connector/connect/common/src/test/resources/query-tests/queries/function_explode.json index 4f8614806c10..35ad40ccdd04 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_explode.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_explode.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_explode.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_explode.proto.bin index c95b243aba77..9c15f942bb11 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_explode.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_explode.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json b/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json index 3ac00c249959..efd7f4b524d4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.proto.bin index 151dce8fff69..9f2cf9554dd1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_explode_outer.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.json b/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.json index 7d18209a0f7c..d425a6de709b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.proto.bin index 09ddfe153431..3c310cb04ce3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_expm1.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_expr.json b/connector/connect/common/src/test/resources/query-tests/queries/function_expr.json index 30c40ef354c9..99c69b8e8905 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_expr.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_expr.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_expr.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_expr.proto.bin index d0ef70d31e7b..2e59d436bc81 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_expr.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_expr.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.json b/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.json index a2f3f83141c1..7f13a1048091 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.proto.bin index 15d31d607402..ac776eb60d2b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_factorial.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_filter.json b/connector/connect/common/src/test/resources/query-tests/queries/function_filter.json index 687f6c0bc23e..f6b565324b8b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_filter.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_filter.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_filter.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_filter.proto.bin index 85831fc0f716..a53c55459866 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_filter.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_filter.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json b/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json index 51ad29afbb69..1d9667c88901 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.proto.bin index a378a54db7fa..5b7db291cc37 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_first.json b/connector/connect/common/src/test/resources/query-tests/queries/function_first.json index e483363cec52..dc33bad3c506 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_first.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_first.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_first.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_first.proto.bin index a52b2870750f..cb029dfd26be 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_first.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_first.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.json b/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.json index 88b318c79acb..32da97271d2d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.proto.bin index 36f77dcb0739..e6bb018a3700 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_flatten.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_floor.json b/connector/connect/common/src/test/resources/query-tests/queries/function_floor.json index 02607b93e476..78924f5f3362 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_floor.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_floor.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_floor.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_floor.proto.bin index 3e9ccb0b2a8d..b52696ca4d00 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_floor.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_floor.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json b/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json index fd092a4e07eb..394621e4dd31 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.proto.bin index c03a3ceb0861..ee0665bab644 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_floor_scale.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_forall.json b/connector/connect/common/src/test/resources/query-tests/queries/function_forall.json index c63c5b5897ae..93134aba0fa9 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_forall.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_forall.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_forall.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_forall.proto.bin index a515a03d9959..3199c758c04a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_forall.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_forall.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.json b/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.json index 89b9807f7a89..daa648c0a599 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.proto.bin index f4263f97b495..81e2c4d5fd54 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_format_number.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.json b/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.json index c23894d1bd15..798e79e6618f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.proto.bin index 8a01afa326f3..8acd3b619b41 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_from_csv.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.json b/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.json index b86b8c129605..ddfa91abca05 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.proto.bin index 23d781bbb92a..ad95d0f2b343 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_from_json.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json b/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json index e6bb43f96282..81d6608adb18 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.proto.bin index 16125102a333..b1c34caaf62f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.json index 8345dfbb6ecf..5d63fd829f30 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.proto.bin index 934b8465135a..34bf9c64f3a9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_get.json b/connector/connect/common/src/test/resources/query-tests/queries/function_get.json index 0980e6fbdca6..7a2a89447c07 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_get.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_get.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_get.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_get.proto.bin index 11f56f6a85aa..be40df955a40 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_get.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_get.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.json b/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.json index 1e2771d53dd3..17adf9230a6e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.proto.bin index b3dc06c1d8f6..08ad8f4f91ba 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_get_json_object.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.json b/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.json index 80d411803e71..bf5d50edec84 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.proto.bin index 3f8f3a5e37d9..44d9d5f8cfb2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_greatest.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hash.json b/connector/connect/common/src/test/resources/query-tests/queries/function_hash.json index 926ca2716c86..6ef504a00645 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_hash.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_hash.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hash.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_hash.proto.bin index ef8fe3f1908c..284700c4c5ea 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_hash.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_hash.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hex.json b/connector/connect/common/src/test/resources/query-tests/queries/function_hex.json index 051304a992a1..af9d0dd29827 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_hex.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_hex.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hex.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_hex.proto.bin index a5961db21338..9d8c3b5e2358 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_hex.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_hex.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hour.json b/connector/connect/common/src/test/resources/query-tests/queries/function_hour.json index b2a3ae643813..2621b9f81913 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_hour.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_hour.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hour.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_hour.proto.bin index 265086f56fee..6cdb50364c13 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_hour.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_hour.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hours.json b/connector/connect/common/src/test/resources/query-tests/queries/function_hours.json index 4cccc0eef477..a72a8656362f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_hours.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_hours.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hours.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_hours.proto.bin index 17cb707448a9..6e8203b89e32 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_hours.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_hours.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.json b/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.json index 3e64ad01515f..2d0d6be0164b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.proto.bin index 4250484e005d..3ad07a2a1ee4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_hypot.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.json b/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.json index 14dd59387b15..896bb3d0209d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.proto.bin index 1c7822a645d5..72df35bd9b38 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_initcap.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_inline.json b/connector/connect/common/src/test/resources/query-tests/queries/function_inline.json index 1e55f7680a2e..4abdac736d0f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_inline.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_inline.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_inline.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_inline.proto.bin index a0210b7cfc17..261e28e3acaa 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_inline.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_inline.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.json b/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.json index a89cbd96e213..d74ee83eeff3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.proto.bin index cc316078c3af..d757e5afe305 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_inline_outer.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json b/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json index 9d3c651874a0..47f2e461eba4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.proto.bin index 4d773db4952f..c3c6414d5d88 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_input_file_name.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.json b/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.json index 0eb722a08ecc..f594918ed930 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.proto.bin index 1f96b82d070b..1030abda5b8c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_isnan.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.json b/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.json index 3d51a2f8748a..7443fc97f42c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.proto.bin index 8abf3ceadc18..3d1fbd4dedfe 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_isnull.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json b/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json index bcded73e4f93..32de63452c36 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.proto.bin index 9b908290bbd2..e51be42b38d3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_json_tuple.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.json b/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.json index 0d987d1252eb..7399d7a6da38 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.proto.bin index ea0ec7f993b2..848a4842e246 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_kurtosis.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lag.json b/connector/connect/common/src/test/resources/query-tests/queries/function_lag.json index ee529f00dc52..dd1cba376f3c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_lag.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_lag.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lag.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_lag.proto.bin index 908b872ec53a..7fd85861fb8c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_lag.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_lag.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_last.json b/connector/connect/common/src/test/resources/query-tests/queries/function_last.json index 428660970ab2..f26e5887ed52 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_last.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_last.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_last.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_last.proto.bin index e14ee792a6c2..69221737be67 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_last.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_last.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.json b/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.json index 21e5d982c608..2cb1635caf47 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.proto.bin index c211151c0a3d..1afb5c02ae34 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_last_day.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lead.json b/connector/connect/common/src/test/resources/query-tests/queries/function_lead.json index 8c38eec6daf8..ef76586d381d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_lead.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_lead.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lead.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_lead.proto.bin index 45f9e784037f..9bcdcdb3617a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_lead.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_lead.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_least.json b/connector/connect/common/src/test/resources/query-tests/queries/function_least.json index 1c83d8187382..403531c9f695 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_least.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_least.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_least.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_least.proto.bin index 9533cf00cf40..c9ead802a961 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_least.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_least.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_length.json b/connector/connect/common/src/test/resources/query-tests/queries/function_length.json index 9c9ed8a6c60c..f2c3c6925589 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_length.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_length.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_length.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_length.proto.bin index c2d143678505..a14f94085b3b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_length.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_length.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json b/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json index 8220a56ba3ff..10caaf184fee 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.proto.bin index 49cdc5adc4fb..75b48541b766 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_levenshtein.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lit.json b/connector/connect/common/src/test/resources/query-tests/queries/function_lit.json index 2ca89e900b8d..3cb1f421207c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_lit.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_lit.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin index 2644023b3725..9149628d7a33 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json index 4c0ae1e1b5aa..68281d2e6d9d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.proto.bin index f793457110fc..b1a9e70c7c80 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_locate.json b/connector/connect/common/src/test/resources/query-tests/queries/function_locate.json index bbf2ef9480dc..7939fdd2c755 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_locate.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_locate.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_locate.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_locate.proto.bin index b1677c5d73a9..cc7ced9957a5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_locate.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_locate.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json b/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json index 996107c534cf..269f39701608 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.proto.bin index f2ab262aaaa8..162ab0108c13 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log.json b/connector/connect/common/src/test/resources/query-tests/queries/function_log.json index b9628776cbbc..1b2d0ed0b144 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_log.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_log.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin index 537849aed11c..548fb480dd27 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log10.json b/connector/connect/common/src/test/resources/query-tests/queries/function_log10.json index 6aa99f1521a9..13292d83c472 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_log10.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_log10.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log10.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_log10.proto.bin index f508b2aeebda..22d4655a6efb 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_log10.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_log10.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.json b/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.json index 359ab79adf3f..4e9e6847c3c3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.proto.bin index f1b7134a1d9c..9a72c377b0cc 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_log1p.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log2.json b/connector/connect/common/src/test/resources/query-tests/queries/function_log2.json index ade59f2f866c..ec29e154a0e1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_log2.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_log2.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log2.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_log2.proto.bin index a7fec7a802f5..34e378065054 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_log2.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_log2.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json b/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json index 185d36b202d8..6bc2a4ec3335 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.proto.bin index 172793b7d465..2e64e15ed555 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_log_with_base.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lower.json b/connector/connect/common/src/test/resources/query-tests/queries/function_lower.json index 34693974e1a4..f7fe5beba2c0 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_lower.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_lower.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lower.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_lower.proto.bin index e3022e48ebec..7c736d93f772 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_lower.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_lower.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.json b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.json index 7d722e655b73..b9f3e6700bfa 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.proto.bin index 71f692f5059d..470ab1cc44ad 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.json b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.json index 0bd0d27b0d1f..aeb39ba09ad2 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,bytes:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.proto.bin index 25d14fd459fc..b4acebb394c7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_lpad_binary.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.json b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.json index a5c4b69cbef0..dd3b45952022 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.proto.bin index 5b78a82e3468..162b6a7337bb 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json index 266ccfa36cea..3c4825792dc3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin index c842be406c87..13455d7091e9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.json b/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.json index 33c1bbcba65e..a363298dd123 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.proto.bin index 8eff6951ba8c..0526825fccad 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_make_date.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map.json index 3f2bb1cf1101..ca9d3bf2bcc7 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map.proto.bin index a774137b38c8..229a48b75131 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.json index 2a1369806c5b..f56f6cee20ab 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.proto.bin index 2b03aedd396a..0a76d3a1193e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_concat.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.json index b7f9c2b1b6bd..56833f965102 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.proto.bin index 6ae21121eb76..e517479020e1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_contains_key.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.json index 2cb5a2af142f..022650654501 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.proto.bin index f8e5894ffe56..f1451d4ad7ba 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_entries.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.json index d49f546bedc7..5099377a52a0 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.proto.bin index cff017d65eb5..fac64e79a5bf 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_filter.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json index 9a999a689ff7..1eb1f7d2ef06 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.proto.bin index 1cc02430683a..f5333b1c882b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json index 0b4983881f0d..399ba8d1021b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.proto.bin index 6ef02d028271..2938c84f7711 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.json index 3719473dfe78..5af013295cd9 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.proto.bin index ed11768367a2..ee19968bacc2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_keys.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.json index b935f371cc45..3c5eb651801d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.proto.bin index df3bf18c8f2e..4cd7c488ada4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_values.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json b/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json index e9206324ed9d..9d035545eb31 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.proto.bin index 28ed3380aebf..f14eb1a3c93d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_max.json b/connector/connect/common/src/test/resources/query-tests/queries/function_max.json index 2aba75d43377..b23dd9d14c64 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_max.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_max.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_max.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_max.proto.bin index 34d523d28b83..788c9539b576 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_max.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_max.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.json b/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.json index dff4f19d5b68..da311e340cc5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.proto.bin index ae2694bd057b..284c2453af8b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_max_by.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_md5.json b/connector/connect/common/src/test/resources/query-tests/queries/function_md5.json index 8147dabc97cb..e8718594b0be 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_md5.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_md5.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_md5.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_md5.proto.bin index 4a962d74d357..d3ec7c26a2ed 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_md5.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_md5.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_median.json b/connector/connect/common/src/test/resources/query-tests/queries/function_median.json index f09852dc5e32..7331454b9ecb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_median.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_median.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_median.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_median.proto.bin index 9e68ef19aae2..59533e5be599 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_median.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_median.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_min.json b/connector/connect/common/src/test/resources/query-tests/queries/function_min.json index 35ab6361cdf8..1b7266b6774e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_min.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_min.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_min.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_min.proto.bin index aa820535cfb1..b82f4c530922 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_min.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_min.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.json b/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.json index 84a56f69c256..d2478f5e81ab 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.proto.bin index a746bab53894..ddc642b95000 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_min_by.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_minute.json b/connector/connect/common/src/test/resources/query-tests/queries/function_minute.json index 12f54f6cf58a..7c749cdff82f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_minute.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_minute.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_minute.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_minute.proto.bin index cda3703a46e1..e81b7dad8533 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_minute.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_minute.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json b/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json index 257a1e6ba7d1..8e8183e9e088 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin index d7fb046953eb..dca0953a387b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json b/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json index e4caa30e4efe..0a14f1008976 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.proto.bin index 04cdd9f2590a..724ce3ac6904 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_month.json b/connector/connect/common/src/test/resources/query-tests/queries/function_month.json index 8065be8dc732..7ea1e5d0375e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_month.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_month.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_month.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_month.proto.bin index a4637b35831e..b97100a6fe2e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_month.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_month.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_months.json b/connector/connect/common/src/test/resources/query-tests/queries/function_months.json index 189def244d2f..278bab76a654 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_months.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_months.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_months.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_months.proto.bin index 9f6ea9641f64..fdcd96750dc9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_months.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_months.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.json b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.json index b5e91be905f6..0fa772d26cd4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.proto.bin index 50ff5c5d88d4..22ddc1813e0f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json index ae7b3f124f9d..d11bfbd7f242 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.proto.bin index 7a5231cfe189..bf9c545911ff 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.json b/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.json index 0af98419ff17..69daab270c2b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.proto.bin index 59f2d8a49f4c..f314a73dcae6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_nanvl.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_negate.json b/connector/connect/common/src/test/resources/query-tests/queries/function_negate.json index 7256a697270d..e269fabe44be 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_negate.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_negate.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_negate.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_negate.proto.bin index 7f45c5e08ee3..9c56c111ceee 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_negate.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_negate.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.json b/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.json index fef8b2fa899c..486523dcad3e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.proto.bin index 3e2716dbc8da..a97bd75f129d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_next_day.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.json b/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.json index 4d14c28dfcf7..4c764a5d5603 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.proto.bin index 51d26e4c70b2..f87e1695f22e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_nth_value.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.json b/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.json index 1cd06b27791e..2346a788b64b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.proto.bin index 6fec68ebb199..d9ccd2e8a600 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_ntile.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.json b/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.json index 1cdb4d9289e3..7be9ac82662a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.proto.bin index 3287d4ee314c..484ebbb6487b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_octet_length.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.json b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.json index 2243bb7f325e..b580570f923a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.proto.bin index 9feb660a45ab..2110ae9c1461 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json index 383b0b8370a1..99d5426c46fb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.proto.bin index 0011bcdb7481..9a09d28d84fd 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json b/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json index 3119e2af8b77..d8778ec8cd81 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.proto.bin index 594de0f59b70..d668f7e1504c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_percent_rank.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json b/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json index 15569edc49e2..6289464de2a3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.proto.bin index 246a2c6b8999..f44ec86888f6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.json b/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.json index e1741769851f..1dc2cb54cbb6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.proto.bin index 36967776636e..a2bb94dbb517 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_pmod.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.json b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.json index 9e6b94da8f7d..f8a9db37e62b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.proto.bin index a4676028ae41..fc50f5f4c85b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json index a272b1cfc6e3..0e8cd4c1509e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.proto.bin index f7f093a0bf49..19d700665e7f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_pow.json b/connector/connect/common/src/test/resources/query-tests/queries/function_pow.json index 5816a7c84604..187636fb360c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_pow.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_pow.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_pow.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_pow.proto.bin index 00e5d023e0fc..6e1d3b06fe87 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_pow.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_pow.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_product.json b/connector/connect/common/src/test/resources/query-tests/queries/function_product.json index 7a3d4c43f9ce..1dfb7f81912d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_product.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_product.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_product.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_product.proto.bin index 21a5feb50f45..8c3fbd31eb6b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_product.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_product.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.json b/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.json index b1554e4647d9..b95867e0be96 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.proto.bin index 2d99708f27c0..fdc2d96fb08c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_quarter.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_radians.json b/connector/connect/common/src/test/resources/query-tests/queries/function_radians.json index f585cf1089ac..837960dedc65 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_radians.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_radians.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_radians.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_radians.proto.bin index a4f6f9531f29..33a2521b22ac 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_radians.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_radians.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.json b/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.json index 454029736d49..5318466706bd 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.proto.bin index efc81b60598e..7fbd33b9869c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_raise_error.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json b/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json index 4532f9fa8507..453ea54bd0ef 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.proto.bin index 7ce12ed0d1ef..566a49d64129 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json b/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json index 88f12466cb7f..ef84f05c3e19 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.proto.bin index 3ba184d4d0c0..b0064842bf30 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rank.json b/connector/connect/common/src/test/resources/query-tests/queries/function_rank.json index ab199e7c6d23..93c8dc38d668 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_rank.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_rank.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rank.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_rank.proto.bin index d15a9c6d0d6c..3aef331fb173 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_rank.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_rank.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json index 154f40365311..5d9c7a5b4a5a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.proto.bin index efaf5e839085..32ba8b6dcb5e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json index 66b3f0b0f6b2..83dd7a8569fd 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.proto.bin index e543d8bbcb8c..b7d3fde25cf8 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.json b/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.json index efb54e0fff8f..93869adfbedc 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.proto.bin index 43bfdf13e3ff..dd7f2d5de513 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_reverse.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rint.json b/connector/connect/common/src/test/resources/query-tests/queries/function_rint.json index 7e86e340854c..ea5bcebf81d7 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_rint.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_rint.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rint.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_rint.proto.bin index 81377729ceb7..bd47adc8476f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_rint.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_rint.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_round.json b/connector/connect/common/src/test/resources/query-tests/queries/function_round.json index c90800492124..585a0befb224 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_round.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_round.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_round.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_round.proto.bin index 90b949a0a680..8625ccb1a58f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_round.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_round.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.json b/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.json index 2185a41e2fbe..3d5ac8afe3db 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.proto.bin index 080d1fc35b8b..90b4fcb27d3f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_row_number.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.json b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.json index 0692f80247a7..d9b78a0cfd7a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.proto.bin index 915898244819..d4c355afee0b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.json b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.json index 0c83e6add70c..0daaf1636f13 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,bytes:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.proto.bin index fd510d0c2411..c6f9f22146c6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_rpad_binary.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.json b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.json index 8f74f50a8b7d..5fe66e8e3359 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.proto.bin index 0c2884acdf5b..4320bf6ac397 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json index be88802d7787..d4c3c0ca68eb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin index 4caf46240c7c..37f4782f4616 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json index 4548b63ee437..6df6438a1a9c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.proto.bin index 130f29db338d..99475ddf30d1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json index e2b8452a5a93..06110d326e1e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.proto.bin index 85260e2e944f..c4ca00e62926 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json index e4017e52169d..ab05ffa940c5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.proto.bin index dd8cc70aec6b..482485501dd3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sec.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sec.json index 9c5b436dfa23..1cab2239755c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sec.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sec.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sec.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sec.proto.bin index a3e2a41d7c36..8760f57a6d4f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sec.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sec.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_second.json b/connector/connect/common/src/test/resources/query-tests/queries/function_second.json index 629782e5bd27..c77a572b88aa 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_second.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_second.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_second.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_second.proto.bin index 91ded71d17bf..193c46e917ba 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_second.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_second.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.json index ac483d09d964..412ac0272dd5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.proto.bin index cdfcb458aed2..4b62f22574d3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.json index 09ab7def5898..991b42faddb7 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.proto.bin index 56360a6c15ec..01c0136c6df1 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sentences_with_locale.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.json index 646c0d974e58..84bced640ff3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.proto.bin index 79a44c8cff86..09e1ab3be7da 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sequence.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.json b/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.json index 21d989e79f34..5c7d953402b2 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.proto.bin index 34da8ae34c62..7f4ee24d5369 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_session_window.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.json index 21f43ef833a8..ce5014ac2f7e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.proto.bin index c5db8db01500..3fdfdb2a072d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sha1.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.json index 03ab5463692e..5278d604e97b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.proto.bin index 8eec0a9be58a..20a0ee1082ae 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sha2.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json index 5f2f9aaec546..12decd300ab0 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.proto.bin index 11f8f737d72f..94bfbc99fce2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftleft.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.json b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.json index 9da302a00611..c2295c4abaaa 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.proto.bin index 6775161f619d..910d12f50d6a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftright.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json index 4565a88d45e9..875e26a5a565 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.proto.bin index 3c13eef3d45a..aba9c425dca9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_signum.json b/connector/connect/common/src/test/resources/query-tests/queries/function_signum.json index 4b29b4ef04ad..bcf6ad7eb174 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_signum.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_signum.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_signum.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_signum.proto.bin index 2fddfe1d63f6..af52abfb7f25 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_signum.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_signum.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sin.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sin.json index 238b7bfb7a36..cb5b0da07345 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sin.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sin.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sin.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sin.proto.bin index 2743ff0a1ec6..a63f574fa59c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sin.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sin.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.json index 5d4f56b387f8..e0f46b428611 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.proto.bin index 13492acdba5b..2f17ab02a6d9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sinh.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_size.json b/connector/connect/common/src/test/resources/query-tests/queries/function_size.json index e75665d8144d..37c9cd1ac1ba 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_size.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_size.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_size.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_size.proto.bin index a0dbe3709cb6..a8ae600a3dd7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_size.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_size.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.json b/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.json index f328e02df8a5..4b14c8d5ca79 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.proto.bin index c4ecdc95ab3a..889f96b2d2a3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_skewness.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_slice.json b/connector/connect/common/src/test/resources/query-tests/queries/function_slice.json index 497a1626eef7..b0a63248784e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_slice.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_slice.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_slice.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_slice.proto.bin index 3e1ff1ec8c8c..620a006f775d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_slice.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_slice.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.json index e8b279c0a936..b42bede5cd17 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.proto.bin index 9fe1405d5d26..994048af2afc 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sort_array.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json b/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json index d07adf80e1a6..851745b32ebe 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.proto.bin index 16dfd3e47555..df99cd64e720 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_split.json b/connector/connect/common/src/test/resources/query-tests/queries/function_split.json index 4fc19aa5811d..001d44dcaaf6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_split.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_split.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_split.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_split.proto.bin index b4b2004d3bc0..cab0bde7b6da 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_split.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_split.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json b/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json index d22760653b08..45a7588838ff 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.proto.bin index 5ed2b1d2ce98..497297fad871 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.json index fe4b6ac5e7fe..f9a2b76520c1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.proto.bin index bd86cdf8ee96..e98e3bdfdb66 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sqrt.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.json b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.json index 766918cadf10..1403817886ca 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.proto.bin index 5558c9428ec5..8d214eea8e74 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json index 4f3e80c8db1d..35e3a08b219f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.proto.bin index 8ae3277a81e9..b679f55014f9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json index dc8450d608e8..17cd0fd5e597 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.proto.bin index cb5ec8062286..9f22eba5e39a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_struct.json b/connector/connect/common/src/test/resources/query-tests/queries/function_struct.json index 4efade2ea124..ba950215a259 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_struct.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_struct.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_struct.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_struct.proto.bin index 62537a07c99f..079c2be3c52e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_struct.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_struct.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_substring.json b/connector/connect/common/src/test/resources/query-tests/queries/function_substring.json index ca8ea713cfb3..84a70cf1c023 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_substring.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_substring.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_substring.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_substring.proto.bin index 9f7b06da99a5..d302cd95c743 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_substring.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_substring.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.json b/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.json index 5d5afc5960d9..dc81d925957c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.proto.bin index bda6160177e9..192bb2e300dc 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_substring_index.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sum.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sum.json index 6735b7a0e162..e9526a20b67f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sum.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sum.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sum.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sum.proto.bin index 47e99c8d52ac..0e347bbc0a16 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sum.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sum.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json b/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json index d4335e2d1ef0..4614cf99ad3a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.proto.bin index 1f1026cbd606..b4cf704391a4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_tan.json b/connector/connect/common/src/test/resources/query-tests/queries/function_tan.json index 73442dc5b694..ead160a7e3ac 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_tan.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_tan.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_tan.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_tan.proto.bin index 5849d81e1891..d674dc033b2c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_tan.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_tan.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.json b/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.json index 6e2571935459..bcd12c664427 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.proto.bin index f17641e49c2f..21c28c3ef88e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_tanh.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json b/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json index 8d5a686db615..e6892d17708b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.proto.bin index c3a483e38701..102afbdda902 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.json b/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.json index e06e9f57e1ec..6b3856f5ac0a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.proto.bin index fb670fcccee7..a3017643a330 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_to_csv.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.json b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.json index 5d21ca960792..8b9d50aa578b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.proto.bin index 6c73117c5369..59178487eef5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json index b2f487085d3d..48ae80d1e70e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.proto.bin index e71ebdd9c600..2641d660ff69 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.json b/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.json index e58801c815eb..7ceeb9d113cd 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.proto.bin index 6cb8d1290aa9..c9461c1aa961 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_to_json.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json index e0f3d6cc0f59..323c57e2ef58 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.proto.bin index 3f854a07963a..ec6bd64f9818 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json index 89b91463b6e1..30f34528319c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.proto.bin index 81ee545465e3..9c2d6d354ca7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json index b826851c2c93..015fbb5cf534 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.proto.bin index 229ab9e87ee1..b2b65089604a 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform.json b/connector/connect/common/src/test/resources/query-tests/queries/function_transform.json index bc8749cc0224..2b357a357731 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_transform.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_transform.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_transform.proto.bin index 73e569681311..44b83a9b98c5 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_transform.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_transform.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.json b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.json index 400c2b0cc86e..0b6a6c24504b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.proto.bin index 97533b22067e..338aa87e0183 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_keys.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.json b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.json index 97a80e484573..71911ab5ed99 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.proto.bin index 7118b0b15a1f..10cf8c503f42 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_values.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json index 7f5384608388..1b296e891bca 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.proto.bin index 41c1878fbf68..86f29399b956 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_translate.json b/connector/connect/common/src/test/resources/query-tests/queries/function_translate.json index 314cef73c2b9..93d155c2857f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_translate.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_translate.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_translate.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_translate.proto.bin index da9da35e74e7..1ce32c8d2843 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_translate.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_translate.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_trim.json b/connector/connect/common/src/test/resources/query-tests/queries/function_trim.json index 7b58028d9043..d2700174bca3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_trim.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_trim.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_trim.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_trim.proto.bin index 7d6b4f38fbf1..d5f4f21510fc 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_trim.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_trim.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json b/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json index cc7d4e103b5c..82b1616ef38e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin index 70f9b0b39a4c..6a86e87c9850 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.json b/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.json index 2270ecf3c1d2..4c596cd86326 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.proto.bin index 132e23ec8039..cdcee95af634 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_trunc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.json b/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.json index e41a3f0433ed..6af2a00ed160 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.proto.bin index fe11d1d7845b..f37ceb91bf42 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_unbase64.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.json b/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.json index dff5f2c2edd5..7c409d023f76 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.proto.bin index 493c70a49f74..fbac2821fdb0 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_unhex.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json index 509a7f909600..e590f7778f2e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.proto.bin index 5db43e06c9af..cb3d967ae012 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.json b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.json index adf3b0a4c1cc..d2e087a5d8a2 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.proto.bin index 54b5642b550a..ddfcdff63d11 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_upper.json b/connector/connect/common/src/test/resources/query-tests/queries/function_upper.json index 91f321d27e83..208ee9231a13 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_upper.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_upper.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_upper.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_upper.proto.bin index 2fc03cc69bbd..5ddbfce96e71 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_upper.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_upper.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.json b/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.json index 4cf7abc8b845..9c74ce4a984f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.proto.bin index 2646cfadba0a..7ca6e8d3b811 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_var_pop.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.json b/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.json index 7077feb5c5ef..979313dd0510 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.proto.bin index 93ac12e8119c..9bd042ad339e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_var_samp.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_variance.json b/connector/connect/common/src/test/resources/query-tests/queries/function_variance.json index 61310a665d65..90a97c3becf4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_variance.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_variance.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_variance.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_variance.proto.bin index fec1298adc2a..fd494fc49639 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_variance.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_variance.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.json b/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.json index b7c1f68937f6..3f46a98569e2 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.proto.bin index c0a1601fc205..ec9b22522360 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_weekofyear.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_window.json b/connector/connect/common/src/test/resources/query-tests/queries/function_window.json index 05322ab65436..bdcb6a398800 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_window.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_window.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_window.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_window.proto.bin index 0226aa0c382a..8cffcc1e9f67 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_window.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_window.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.json b/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.json index de6b0a38fda2..4809ea21261c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.json @@ -1,8 +1,17 @@ { + "common": { + "planId": "2" + }, "project": { "input": { + "common": { + "planId": "1" + }, "withColumns": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } @@ -10,7 +19,8 @@ "aliases": [{ "expr": { "unresolvedAttribute": { - "unparsedIdentifier": "wt" + "unparsedIdentifier": "wt", + "planId": "0" } }, "name": ["wt"], diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.proto.bin index e39dd78802c1..c143520df08c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_window_time.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json b/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json index fac2a4ed0be7..c20739d09ff1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.proto.bin index 6ed2e2a243a8..414c76fc5ce7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_xxhash64.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_year.json b/connector/connect/common/src/test/resources/query-tests/queries/function_year.json index 502a7b5f1dc4..b8a4ee5a1652 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_year.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_year.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_year.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_year.proto.bin index 74f3e61f8167..623bc9ac6d81 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_year.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_year.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_years.json b/connector/connect/common/src/test/resources/query-tests/queries/function_years.json index a0b6f4228d06..2e8730732027 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_years.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_years.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_years.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_years.proto.bin index f1e2a949fb44..30c25423fd56 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_years.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_years.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.json b/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.json index a7c603ed6bbb..d1d0e7293c8f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double,d:struct\u003cid:bigint,a:int,b:double\u003e,e:array\u003cint\u003e,f:map\u003cstring,struct\u003cid:bigint,a:int,b:double\u003e\u003e,g:string\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.proto.bin index 2d6bec136a7c..c9a6dff84736 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/function_zip_with.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.json index 4cf2ae0c8c28..4a1cfddb0288 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } @@ -16,7 +22,8 @@ "functionName": "max", "arguments": [{ "unresolvedAttribute": { - "unparsedIdentifier": "a" + "unparsedIdentifier": "a", + "planId": "0" } }] } @@ -25,7 +32,8 @@ "functionName": "stddev", "arguments": [{ "unresolvedAttribute": { - "unparsedIdentifier": "b" + "unparsedIdentifier": "b", + "planId": "0" } }] } @@ -34,7 +42,8 @@ "functionName": "stddev", "arguments": [{ "unresolvedAttribute": { - "unparsedIdentifier": "b" + "unparsedIdentifier": "b", + "planId": "0" } }] } @@ -43,7 +52,8 @@ "functionName": "avg", "arguments": [{ "unresolvedAttribute": { - "unparsedIdentifier": "b" + "unparsedIdentifier": "b", + "planId": "0" } }] } @@ -52,7 +62,8 @@ "functionName": "avg", "arguments": [{ "unresolvedAttribute": { - "unparsedIdentifier": "b" + "unparsedIdentifier": "b", + "planId": "0" } }] } @@ -61,7 +72,8 @@ "functionName": "avg", "arguments": [{ "unresolvedAttribute": { - "unparsedIdentifier": "b" + "unparsedIdentifier": "b", + "planId": "0" } }] } @@ -78,7 +90,8 @@ "functionName": "count", "arguments": [{ "unresolvedAttribute": { - "unparsedIdentifier": "a" + "unparsedIdentifier": "a", + "planId": "0" } }] } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.proto.bin index eed57649c452..cfd6c2daa84b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.json index fd2264fd2aec..e61616786158 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.proto.bin index b12dd5229db4..d6daa1cc31f7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_agg_columns.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.json index df4216bdd519..5785eee2cadb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.proto.bin index 33cbb49f1fe4..4a18ea2d82d9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_avg.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.json index c28c167f21b1..f92e22493e07 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.proto.bin index d3920650eb5f..5bb539195df9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_count.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.json index 262232063dd1..3225a475a9b3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.proto.bin index e43c9e3e325d..651274b1afca 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_max.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.json index df4216bdd519..5785eee2cadb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.proto.bin index 33cbb49f1fe4..4a18ea2d82d9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_mean.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.json index c6c55dde8b47..afcc07d2c869 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.proto.bin index c7ad1785cd8b..6e038bf0b315 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_min.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.json b/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.json index 48d3820dea93..74dd5b045aa5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.proto.bin index 673b74527033..fe2451ca18fb 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/groupby_sum.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/hint.json b/connector/connect/common/src/test/resources/query-tests/queries/hint.json index 38f3ff1ab7a0..bb5b848b744d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/hint.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/hint.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "hint": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/hint.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/hint.proto.bin index 8832794f792a..8eb4f4120351 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/hint.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/hint.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/intersect.json b/connector/connect/common/src/test/resources/query-tests/queries/intersect.json index cd8167116c0c..f290397c55ca 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/intersect.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/intersect.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "setOp": { "leftInput": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "rightInput": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/intersect.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/intersect.proto.bin index 71816354cda7..0ea7edc5cee3 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/intersect.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/intersect.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.json b/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.json index 9fd2a7a37275..d8fe5fe0b7e7 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "setOp": { "leftInput": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "rightInput": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.proto.bin index 5e3325c73c13..6df2125682bc 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/intersectAll.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_condition.json b/connector/connect/common/src/test/resources/query-tests/queries/join_condition.json index 94abd7a159f1..993cd98a7dd1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_condition.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_condition.json @@ -1,8 +1,17 @@ { + "common": { + "planId": "4" + }, "join": { "left": { + "common": { + "planId": "1" + }, "subqueryAlias": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } @@ -11,8 +20,14 @@ } }, "right": { + "common": { + "planId": "3" + }, "subqueryAlias": { "input": { + "common": { + "planId": "2" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_condition.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_condition.proto.bin index 84ddc108103c..1d11fe5e75bc 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_condition.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_condition.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json index 36e203a42561..527338c56ae6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json @@ -1,8 +1,17 @@ { + "common": { + "planId": "4" + }, "join": { "left": { + "common": { + "planId": "1" + }, "subqueryAlias": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } @@ -11,8 +20,14 @@ } }, "right": { + "common": { + "planId": "3" + }, "subqueryAlias": { "input": { + "common": { + "planId": "2" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.proto.bin index c1cc916d3275..5d3de55da9cf 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_condition.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.json b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.json index 0308a128db34..8c53a193162d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.proto.bin index 9a269059bf74..44bf1a6793cd 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_no_condition.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.json b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.json index 9f9f1a0cf30e..42b4eec5d9f1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.proto.bin index 0e4192d7afce..98e2a4fe9b58 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.json b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.json index 9f9f1a0cf30e..42b4eec5d9f1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.proto.bin index 0e4192d7afce..98e2a4fe9b58 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.json b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.json index b5137f978a9c..2c2bde49b190 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.proto.bin index 0708749bd473..7d4a1aeb11ef 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_inner_using_single_col.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.json b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.json index 4bd0b9ec2c7e..9b592426cf96 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.proto.bin index 954128f8c1e8..4c4b6ecb2076 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.json b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.json index 3d06f7ab31f9..3f1c46f08e81 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.proto.bin index 5878e776d87b..2a5410fc0631 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.json b/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.json index b5b8b3a40f9d..46f144de61a9 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "join": { "left": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "right": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.proto.bin index 0c72726a8c22..c2fa60619d70 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/join_using_single_col.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/limit.json b/connector/connect/common/src/test/resources/query-tests/queries/limit.json index e755944a82e4..acf01c196891 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/limit.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/limit.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "limit": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/limit.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/limit.proto.bin index 71249c1f2c97..f3f4771fe4de 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/limit.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/limit.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.json b/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.json index 292adb11b17e..12db0a5abe36 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "unpivot": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.proto.bin index dc1a40f8d488..23a6aa1289a9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/melt_no_values.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/melt_values.json b/connector/connect/common/src/test/resources/query-tests/queries/melt_values.json index 79d494c3c5d6..e2a004f46e78 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/melt_values.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/melt_values.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "unpivot": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/melt_values.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/melt_values.proto.bin index 1ac7cb290f84..e021e1110def 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/melt_values.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/melt_values.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/offset.json b/connector/connect/common/src/test/resources/query-tests/queries/offset.json index e03d5072ea40..80796160b96d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/offset.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/offset.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "offset": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/offset.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/offset.proto.bin index 8c7dde8f92c9..6671eebb9318 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/offset.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/offset.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json index d935be14ee65..72ea72d79549 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin index bebe09e64a91..00fa9f8b5c02 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json index 22508b999d38..e7f63a15c288 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin index 3e7b74ea6b59..a907e66a130d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.json b/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.json index 99268661e728..5ceb1d5a0871 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "sql": { "query": "select 1", "args": { diff --git a/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.proto.bin index fd9304b4e47c..50bc8457f317 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/parameterized_sql.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/range.json b/connector/connect/common/src/test/resources/query-tests/queries/range.json index b9d83e3e5940..8afa44fac6cf 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/range.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/range.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "range": { "start": "1", "end": "10", diff --git a/connector/connect/common/src/test/resources/query-tests/queries/range.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/range.proto.bin index 717b9adab666..277a02cea558 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/range.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/range.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read.json b/connector/connect/common/src/test/resources/query-tests/queries/read.json index f5ffb3c961b5..d5580c1321ec 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/read.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/read.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "dataSource": { "format": "csv", diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/read.proto.bin index ede57af1130c..c50391bb1a8f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/read.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/read.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_csv.json b/connector/connect/common/src/test/resources/query-tests/queries/read_csv.json index 6095d200f628..ec1eed1c6cf3 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/read_csv.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/read_csv.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "dataSource": { "format": "csv", diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_csv.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/read_csv.proto.bin index 9bbab0fe2af6..d8b5ca93f2f7 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/read_csv.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/read_csv.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_json.json b/connector/connect/common/src/test/resources/query-tests/queries/read_json.json index 2e2f83d11915..63dadc129dc8 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/read_json.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/read_json.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "dataSource": { "format": "json", diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_json.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/read_json.proto.bin index 22557aca38d6..1d829df6bbcf 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/read_json.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/read_json.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_orc.json b/connector/connect/common/src/test/resources/query-tests/queries/read_orc.json index caa6c951d3ea..b78d7d6ecd61 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/read_orc.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/read_orc.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "dataSource": { "format": "orc", diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_orc.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/read_orc.proto.bin index 95d07fd0a9b0..6a67db561dc8 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/read_orc.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/read_orc.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.json b/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.json index 05d799fd9cb0..0a201a43c744 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "dataSource": { "format": "parquet", diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.proto.bin index 5fc1954428f8..f16b28dcce01 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/read_parquet.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_table.json b/connector/connect/common/src/test/resources/query-tests/queries/read_table.json index 634310c27ff2..b2cd4ae0a5ba 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/read_table.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/read_table.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "namedTable": { "unparsedIdentifier": "myTable" diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_table.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/read_table.proto.bin index f6ffaf988e7d..956da78861d0 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/read_table.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/read_table.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_text.json b/connector/connect/common/src/test/resources/query-tests/queries/read_text.json index 8dc1e26a70ec..de7a306a52fb 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/read_text.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/read_text.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "dataSource": { "format": "text", diff --git a/connector/connect/common/src/test/resources/query-tests/queries/read_text.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/read_text.proto.bin index 97d167f6e2e9..3f3bbf8769c4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/read_text.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/read_text.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartition.json b/connector/connect/common/src/test/resources/query-tests/queries/repartition.json index 770b227b7072..163742886c3a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/repartition.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/repartition.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "repartition": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartition.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/repartition.proto.bin index 03bb47876176..5265e0e6175c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/repartition.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/repartition.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.json b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.json index deb4c6188859..98bd4c988abc 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "repartitionByExpression": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.proto.bin index 531737b7548b..8ee220833d9e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.json b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.json index eede8a4b1acd..604d0330fedd 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "repartitionByExpression": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.proto.bin index 8139f2cb3977..a3f1546cca1f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.json b/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.json index c91a30eb0e9c..81113afea353 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "repartitionByExpression": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.proto.bin index c217e9d9d93f..50ff8c590cda 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/repartition_expressions.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.json b/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.json index d70380d12286..996beda2253a 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "repartitionByExpression": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.proto.bin index 47b3ab9daf53..73e22f120ed9 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.json b/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.json index f976e4ea10f8..1102db18830b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.proto.bin index 89ef8ff947b4..64dbb597c365 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/rollup_column.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.json b/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.json index f976e4ea10f8..1102db18830b 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "aggregate": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.proto.bin index 89ef8ff947b4..64dbb597c365 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/rollup_string.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.json b/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.json index 3f4dbedac3ac..88e80a3f60c6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sample": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.proto.bin index 7754a5e213d2..546c9c9c69ca 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sample_fraction_seed.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.json b/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.json index 5f9f6f219669..75d3b2421601 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sample": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.proto.bin index 2e1efe2e5276..48650897e676 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select.json b/connector/connect/common/src/test/resources/query-tests/queries/select.json index 8d4b511b1f93..8ef46a6cc2aa 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/select.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/select.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/select.proto.bin index e926353d8363..2bc4bd85a580 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/select.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/select.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.json b/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.json index b38c071d8230..9c2815cffb75 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.proto.bin index 9f203955a8de..88824d7f896f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/selectExpr.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/select_strings.json index cde5f0721a34..421b9aa12001 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/select_strings.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/select_strings.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_strings.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/select_strings.proto.bin index 507d3c15e613..f868b46f3e58 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/select_strings.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/select_strings.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.json b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.json index 08d98935f400..c45a326a01b4 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.proto.bin index 5c076fa9d120..49e24e6f6f22 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.json index 6b13151641d5..dcded7cb32d8 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.proto.bin index 572abd9c7a0f..f5ff32982388 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json index 65354a131301..76b4d92d71c1 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin index f3f7a40246b2..9c059d244aec 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sort_columns.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json index 6d127fa5c3c7..7955221d7d78 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "sort": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin index 1c7568f4f34c..e780d351c8af 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sort_strings.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sql.json b/connector/connect/common/src/test/resources/query-tests/queries/sql.json index c36812b9791e..c4bc9b2c0827 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/sql.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/sql.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "sql": { "query": "select 1" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/sql.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/sql.proto.bin index 92499c516c31..3d4394f23af0 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/sql.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/sql.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/summary.json b/connector/connect/common/src/test/resources/query-tests/queries/summary.json index 48c336fbf7c4..cbfe9bcf7b08 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/summary.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/summary.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "summary": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/summary.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/summary.proto.bin index 69363710ef2e..a88d61cdc76b 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/summary.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/summary.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/table.json b/connector/connect/common/src/test/resources/query-tests/queries/table.json index 634310c27ff2..b2cd4ae0a5ba 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/table.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/table.json @@ -1,4 +1,7 @@ { + "common": { + "planId": "0" + }, "read": { "namedTable": { "unparsedIdentifier": "myTable" diff --git a/connector/connect/common/src/test/resources/query-tests/queries/table.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/table.proto.bin index f6ffaf988e7d..956da78861d0 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/table.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/table.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/to.json b/connector/connect/common/src/test/resources/query-tests/queries/to.json index e09913ff5626..a3e07202c106 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/to.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/to.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "toSchema": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/to.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/to.proto.bin index ce3c15096736..8e15aa6c2791 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/to.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/to.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/toDF.json b/connector/connect/common/src/test/resources/query-tests/queries/toDF.json index e753b7d0e3af..8111bc76a8a8 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/toDF.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/toDF.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "toDf": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/toDF.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/toDF.proto.bin index b88bdf99169d..3238291e8794 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/toDF.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/toDF.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/union.json b/connector/connect/common/src/test/resources/query-tests/queries/union.json index 170e0f09cf98..9048133ca638 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/union.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/union.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "setOp": { "leftInput": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "rightInput": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/union.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/union.proto.bin index 7c4f869e44fc..caafd1ef998d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/union.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/union.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unionAll.json b/connector/connect/common/src/test/resources/query-tests/queries/unionAll.json index 170e0f09cf98..9048133ca638 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/unionAll.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/unionAll.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "setOp": { "leftInput": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "rightInput": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unionAll.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/unionAll.proto.bin index 7c4f869e44fc..caafd1ef998d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/unionAll.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/unionAll.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unionByName.json b/connector/connect/common/src/test/resources/query-tests/queries/unionByName.json index 9446f0127e9a..9244eb08790d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/unionByName.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/unionByName.json @@ -1,8 +1,17 @@ { + "common": { + "planId": "4" + }, "setOp": { "leftInput": { + "common": { + "planId": "1" + }, "drop": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } @@ -15,8 +24,14 @@ } }, "rightInput": { + "common": { + "planId": "3" + }, "drop": { "input": { + "common": { + "planId": "2" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unionByName.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/unionByName.proto.bin index 522c1bb0a16d..64d9fb901d21 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/unionByName.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/unionByName.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.json b/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.json index 70e770c135fd..98870ffe7175 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.json @@ -1,11 +1,20 @@ { + "common": { + "planId": "2" + }, "setOp": { "leftInput": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } }, "rightInput": { + "common": { + "planId": "1" + }, "localRelation": { "schema": "struct\u003ca:int,id:bigint,payload:binary\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.proto.bin index 29e0896b6c63..4facbbc553ea 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.json b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.json index 662328af9a75..9f550c031914 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "unpivot": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.proto.bin index 9f2600404f20..ac3bad8bd04e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_no_values.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.json b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.json index 3557958fd29d..92bc19d195c6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "unpivot": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.proto.bin index 9fb45c9c0c19..7f717cb23517 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/unpivot_values.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/where_column.json b/connector/connect/common/src/test/resources/query-tests/queries/where_column.json index f8ec863648c9..bef80a7e6ed5 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/where_column.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/where_column.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "filter": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/where_column.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/where_column.proto.bin index eae5db282cae..e472ed0715b6 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/where_column.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/where_column.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/where_expr.json b/connector/connect/common/src/test/resources/query-tests/queries/where_expr.json index c244ff0e8910..dc7523bcaade 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/where_expr.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/where_expr.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "filter": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/where_expr.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/where_expr.proto.bin index 63b44b32b634..380a1763b81e 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/where_expr.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/where_expr.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/window.json b/connector/connect/common/src/test/resources/query-tests/queries/window.json index 8649a9d6e543..23fd5c1556ec 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/window.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/window.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "project": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/window.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/window.proto.bin index caa9d66934c2..a89c0d6a6a3f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/window.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/window.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.json b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.json index ba7d76aec996..731cf844afe6 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "withColumnsRenamed": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.proto.bin index 38130a873d82..64fcf7855ecb 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.json b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.json index ee507379fa41..570bfa32233d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "withColumnsRenamed": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.proto.bin index 5a8cfaa51ee4..42df8ea1d111 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.json b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.json index 8d0bfc254eb7..23b2e1d41d3c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "withColumnsRenamed": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.proto.bin index 5c60a64d3ad4..f46d01646c6f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/withColumnRenamed_single.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.json b/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.json index 5bf8b01d5c43..8863d15f2764 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "withColumns": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.proto.bin index f6693d6b8b8a..6d53a883a5f4 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/withColumn_single.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.json b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.json index 21e75d07aef3..a59f4abd47ce 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "withColumns": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.proto.bin index f71f62489598..be381f62594c 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_java_map.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.json b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.json index aca9ce7db7d6..99405a73041f 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "withColumns": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.proto.bin index 52a22ed9cecb..77ee1900e73f 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/withColumns_scala_map.proto.bin differ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.json b/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.json index 0f28e7655a4a..6ba7e5cd55bd 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.json @@ -1,6 +1,12 @@ { + "common": { + "planId": "1" + }, "withColumns": { "input": { + "common": { + "planId": "0" + }, "localRelation": { "schema": "struct\u003cid:bigint,a:int,b:double\u003e" } @@ -8,7 +14,8 @@ "aliases": [{ "expr": { "unresolvedAttribute": { - "unparsedIdentifier": "id" + "unparsedIdentifier": "id", + "planId": "0" } }, "name": ["id"], diff --git a/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.proto.bin index e9aa65874e4f..f814b37d0ac2 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/withMetadata.proto.bin differ