diff --git a/.github/actions/rust-test/action.yaml b/.github/actions/rust-test/action.yaml index b543bcf00..3aeba0666 100644 --- a/.github/actions/rust-test/action.yaml +++ b/.github/actions/rust-test/action.yaml @@ -37,6 +37,12 @@ runs: cd native cargo check --benches + - name: Check unused dependencies + shell: bash + run: | + cd native + cargo install cargo-machete && cargo machete + - name: Cache Maven dependencies uses: actions/cache@v4 with: diff --git a/.github/workflows/miri.yml b/.github/workflows/miri.yml new file mode 100644 index 000000000..a07ecc35e --- /dev/null +++ b/.github/workflows/miri.yml @@ -0,0 +1,49 @@ +# 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. + +name: Run Miri Safety Checks + +on: + push: + paths-ignore: + - "doc/**" + - "docs/**" + - "**.md" + pull_request: + paths-ignore: + - "doc/**" + - "docs/**" + - "**.md" + # manual trigger + # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow + workflow_dispatch: + +jobs: + miri: + name: "Miri" + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Install Miri + run: | + rustup toolchain install nightly --component miri + rustup override set nightly + cargo miri setup + - name: Test with Miri + run: | + cd native + MIRIFLAGS="-Zmiri-disable-isolation" cargo miri test diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 787e8b4e9..bcc8d25a7 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -286,9 +286,10 @@ object CometConf extends ShimCometConf { conf("spark.comet.explainFallback.enabled") .doc( "When this setting is enabled, Comet will provide logging explaining the reason(s) " + - "why a query stage cannot be executed natively.") + "why a query stage cannot be executed natively. Set this to false to " + + "reduce the amount of logging.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val COMET_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.batchSize") .doc("The columnar batch size, i.e., the maximum number of rows that a batch can contain.") diff --git a/common/src/main/scala/org/apache/comet/package.scala b/common/src/main/scala/org/apache/comet/package.scala index c9aca7538..f44139ba6 100644 --- a/common/src/main/scala/org/apache/comet/package.scala +++ b/common/src/main/scala/org/apache/comet/package.scala @@ -21,8 +21,19 @@ package org.apache import java.util.Properties +import org.apache.arrow.memory.RootAllocator + package object comet { + /** + * The root allocator for Comet execution. Because Arrow Java memory management is based on + * reference counting, exposed arrays increase the reference count of the underlying buffers. + * Until the reference count is zero, the memory will not be released. If the consumer side is + * finished later than the close of the allocator, the allocator will think the memory is + * leaked. To avoid this, we use a single allocator for the whole execution process. + */ + val CometArrowAllocator = new RootAllocator(Long.MaxValue) + /** * Provides access to build information about the Comet libraries. This will be used by the * benchmarking software to provide the source revision and repository. In addition, the build diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala index 595c0a427..89f79c9cd 100644 --- a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -22,18 +22,18 @@ package org.apache.comet.vector import scala.collection.mutable import org.apache.arrow.c.{ArrowArray, ArrowImporter, ArrowSchema, CDataDictionaryProvider, Data} -import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.dictionary.DictionaryProvider import org.apache.spark.SparkException import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.comet.CometArrowAllocator + class NativeUtil { import Utils._ - private val allocator = new RootAllocator(Long.MaxValue) - .newChildAllocator(this.getClass.getSimpleName, 0, Long.MaxValue) + private val allocator = CometArrowAllocator private val dictionaryProvider: CDataDictionaryProvider = new CDataDictionaryProvider private val importer = new ArrowImporter(allocator) diff --git a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala index 4a08f0521..b8106a96e 100644 --- a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala +++ b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala @@ -21,20 +21,20 @@ package org.apache.comet.vector import java.nio.channels.ReadableByteChannel -import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.ipc.{ArrowStreamReader, ReadChannel} import org.apache.arrow.vector.ipc.message.MessageChannelReader import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.comet.CometArrowAllocator + /** * A reader that consumes Arrow data from an input channel, and produces Comet batches. */ case class StreamReader(channel: ReadableByteChannel, source: String) extends AutoCloseable { - private var allocator = new RootAllocator(Long.MaxValue) - .newChildAllocator(s"${this.getClass.getSimpleName}/$source", 0, Long.MaxValue) - private val channelReader = new MessageChannelReader(new ReadChannel(channel), allocator) - private var arrowReader = new ArrowStreamReader(channelReader, allocator) + private val channelReader = + new MessageChannelReader(new ReadChannel(channel), CometArrowAllocator) + private var arrowReader = new ArrowStreamReader(channelReader, CometArrowAllocator) private var root = arrowReader.getVectorSchemaRoot def nextBatch(): Option[ColumnarBatch] = { @@ -53,11 +53,9 @@ case class StreamReader(channel: ReadableByteChannel, source: String) extends Au if (root != null) { arrowReader.close() root.close() - allocator.close() arrowReader = null root = null - allocator = null } } } diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 8129c7455..3c88d6a7e 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2491,8 +2491,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/3.5.1.diff b/dev/diffs/3.5.1.diff index ebc4cdb8c..cf6754416 100644 --- a/dev/diffs/3.5.1.diff +++ b/dev/diffs/3.5.1.diff @@ -2650,8 +2650,8 @@ index dd55fcfe42c..293e9dc2986 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 4031015df..abd6e9a04 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -415,7 +415,7 @@ index 16a493b5290..3f0b70e2d59 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index 2c24cc7d570..50a2ce86117 100644 +index 2c24cc7d570..d46dc5e138a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -442,7 +442,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("partition pruning in broadcast hash joins with aliases") { + test("partition pruning in broadcast hash joins with aliases", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { Given("alias with simple join condition, using attribute names only") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( @@ -452,7 +452,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("partition pruning in broadcast hash joins") { + test("partition pruning in broadcast hash joins", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { Given("disable broadcast pruning and disable subquery duplication") withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", @@ -462,7 +462,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("different broadcast subqueries with identical children") { + test("different broadcast subqueries with identical children", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( @@ -492,7 +492,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-32817: DPP throws error when the broadcast side is empty") { + test("SPARK-32817: DPP throws error when the broadcast side is empty", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", @@ -502,7 +502,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning") { + test("SPARK-36444: Remove OptimizeSubqueries from batch of PartitionPruning", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ @@ -521,7 +521,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") { + test("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withTable("duplicate_keys") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { Seq[(Int, String)]((1, "NL"), (1, "NL"), (3, "US"), (3, "US"), (3, "US")) @@ -531,7 +531,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty") { + test("SPARK-39338: Remove dynamic pruning subquery if pruningKey's references is empty", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ @@ -541,7 +541,7 @@ index 2c24cc7d570..50a2ce86117 100644 - test("SPARK-39217: Makes DPP support the pruning side has Union") { + test("SPARK-39217: Makes DPP support the pruning side has Union", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ @@ -1159,7 +1159,7 @@ index 15de4c5cc5b..6a85dfb6883 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index b5bac8079c4..544c1ddc697 100644 +index b5bac8079c4..a3731888e12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,8 @@ @@ -1190,7 +1190,7 @@ index b5bac8079c4..544c1ddc697 100644 - test("join with ordering requirement") { + test("join with ordering requirement", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { val query = "select * from (select key, a, c, b from testView) as t1 join " + "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" assertProjectExec(query, 2, 2) @@ -2590,7 +2590,7 @@ index af07aceaed1..ed0b5e6d9be 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index 5fbf379644f..32711763ec1 100644 +index 5fbf379644f..6153046a787 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -2651,8 +2651,8 @@ index 5fbf379644f..32711763ec1 100644 val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child -+ case CometFilterExec(_, _, _, child, _) => child -+ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child ++ case CometFilterExec(_, _, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, _, child, _), _) => child } spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 283749076..501a2ba3c 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -39,7 +39,7 @@ Comet provides the following configuration settings. | spark.comet.exec.shuffle.enabled | Whether to enable Comet native shuffle. By default, this config is false. Note that this requires setting 'spark.shuffle.manager' to 'org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager'. 'spark.shuffle.manager' must be set before starting the Spark application and cannot be changed during the application. | false | | spark.comet.exec.shuffle.mode | The mode of Comet shuffle. This config is only effective if Comet shuffle is enabled. Available modes are 'native', 'jvm', and 'auto'. 'native' is for native shuffle which has best performance in general. 'jvm' is for jvm-based columnar shuffle which has higher coverage than native shuffle. 'auto' is for Comet to choose the best shuffle mode based on the query plan. By default, this config is 'jvm'. | jvm | | spark.comet.explain.verbose.enabled | When this setting is enabled, Comet will provide a verbose tree representation of the extended information. | false | -| spark.comet.explainFallback.enabled | When this setting is enabled, Comet will provide logging explaining the reason(s) why a query stage cannot be executed natively. | false | +| spark.comet.explainFallback.enabled | When this setting is enabled, Comet will provide logging explaining the reason(s) why a query stage cannot be executed natively. Set this to false to reduce the amount of logging. | true | | spark.comet.memory.overhead.factor | Fraction of executor memory to be allocated as additional non-heap memory per executor process for Comet. Default value is 0.2. | 0.2 | | spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB. | 402653184b | | spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | diff --git a/native/Cargo.lock b/native/Cargo.lock index ccb6433d9..f73f28629 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -114,8 +114,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6127ea5e585a12ec9f742232442828ebaf264dfa5eefdd71282376c599562b77" dependencies = [ "arrow-arith", "arrow-array", @@ -134,8 +135,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7add7f39210b7d726e2a8efc0083e7bf06e8f2d15bdb4896b564dce4410fbf5d" dependencies = [ "arrow-array", "arrow-buffer", @@ -148,8 +150,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81c16ec702d3898c2f5cfdc148443c6cd7dbe5bac28399859eb0a3d38f072827" dependencies = [ "ahash", "arrow-buffer", @@ -164,8 +167,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cae6970bab043c4fbc10aee1660ceb5b306d0c42c8cc5f6ae564efcd9759b663" dependencies = [ "bytes", "half", @@ -174,8 +178,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c7ef44f26ef4f8edc392a048324ed5d757ad09135eff6d5509e6450d39e0398" dependencies = [ "arrow-array", "arrow-buffer", @@ -194,8 +199,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f843490bd258c5182b66e888161bb6f198f49f3792f7c7f98198b924ae0f564" dependencies = [ "arrow-array", "arrow-buffer", @@ -212,8 +218,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a769666ffac256dd301006faca1ca553d0ae7cffcf4cd07095f73f95eb226514" dependencies = [ "arrow-buffer", "arrow-schema", @@ -223,8 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbf9c3fb57390a1af0b7bb3b5558c1ee1f63905f3eccf49ae7676a8d1e6e5a72" dependencies = [ "arrow-array", "arrow-buffer", @@ -237,8 +245,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "654e7f3724176b66ddfacba31af397c48e106fbe4d281c8144e7d237df5acfd7" dependencies = [ "arrow-array", "arrow-buffer", @@ -256,8 +265,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8008370e624e8e3c68174faaf793540287106cfda8ad1da862fdc53d8e096b4" dependencies = [ "arrow-array", "arrow-buffer", @@ -270,8 +280,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca5e3a6b7fda8d9fe03f3b18a2d946354ea7f3c8e4076dbdb502ad50d9d44824" dependencies = [ "ahash", "arrow-array", @@ -284,16 +295,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dab1c12b40e29d9f3b699e0203c2a73ba558444c05e388a4377208f8f9c97eee" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "arrow-select" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e80159088ffe8c48965cb9b1a7c968b2729f29f37363df7eca177fc3281fe7c3" dependencies = [ "ahash", "arrow-array", @@ -305,8 +318,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" +version = "52.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fd04a6ea7de183648edbcb7a6dd925bbd04c210895f6384c780e27a9b54afcd" dependencies = [ "arrow-array", "arrow-buffer", @@ -327,13 +341,13 @@ checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" [[package]] name = "async-trait" -version = "0.1.80" +version = "0.1.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -468,9 +482,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.104" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74b6a57f98764a267ff415d50a25e6e166f3831a5071af4995296ea97d210490" +checksum = "907d8581360765417f8f2e0e7d602733bbed60156b4465b7617243689ef9b83d" dependencies = [ "jobserver", "libc", @@ -574,18 +588,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.8" +version = "4.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84b3edb18336f4df585bc9aa31dd99c036dfa5dc5e9a2939a722a188f3a8970d" +checksum = "64acc1846d54c1fe936a78dc189c34e28d3f5afc348403f28ecf53660b9b8462" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.5.8" +version = "4.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1c09dd5ada6c6c78075d6fd0da3f90d8080651e2d6cc8eb2f1aaa4034ced708" +checksum = "6fb8393d67ba2e7bfaf28a23458e4e2b543cc73a99595511eb207fdb8aede942" dependencies = [ "anstyle", "clap_lex", @@ -790,8 +804,8 @@ dependencies = [ [[package]] name = "datafusion" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -843,26 +857,22 @@ dependencies = [ "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-string", "assertables", "async-trait", "brotli", - "byteorder", "bytes", "chrono", - "chrono-tz 0.8.6", "crc32fast", "criterion", "datafusion", + "datafusion-comet-spark-expr", "datafusion-common", "datafusion-expr", - "datafusion-functions", "datafusion-physical-expr", "datafusion-physical-expr-common", "flate2", "futures", "half", - "hashbrown", "hex", "itertools 0.11.0", "jni", @@ -873,9 +883,7 @@ dependencies = [ "mimalloc", "num", "once_cell", - "parking_lot", "parquet", - "parquet-format", "paste", "pprof", "prost 0.12.6", @@ -887,18 +895,36 @@ dependencies = [ "snap", "tempfile", "thiserror", - "thrift 0.17.0", "tokio", - "tokio-stream", "twox-hash", "unicode-segmentation", "zstd", ] +[[package]] +name = "datafusion-comet-spark-expr" +version = "0.1.0" +dependencies = [ + "arrow", + "arrow-array", + "arrow-schema", + "chrono", + "chrono-tz 0.8.6", + "datafusion", + "datafusion-common", + "datafusion-expr", + "datafusion-functions", + "datafusion-physical-expr", + "datafusion-physical-plan", + "num", + "regex", + "thiserror", +] + [[package]] name = "datafusion-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -917,16 +943,16 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "tokio", ] [[package]] name = "datafusion-execution" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "chrono", @@ -945,8 +971,8 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -963,8 +989,8 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "base64", @@ -974,7 +1000,6 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-physical-expr", "hashbrown", "hex", "itertools 0.12.1", @@ -989,8 +1014,8 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -1006,8 +1031,8 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "async-trait", @@ -1019,13 +1044,14 @@ dependencies = [ "indexmap", "itertools 0.12.1", "log", + "paste", "regex-syntax", ] [[package]] name = "datafusion-physical-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -1039,7 +1065,6 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate", "datafusion-physical-expr-common", "half", "hashbrown", @@ -1054,19 +1079,21 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ + "ahash", "arrow", "datafusion-common", "datafusion-expr", + "hashbrown", "rand", ] [[package]] name = "datafusion-physical-plan" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "ahash", "arrow", @@ -1098,8 +1125,8 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" +version = "40.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=40.0.0#4cae81363e29f011c6602a7a7a54e1aaee841046" dependencies = [ "arrow", "arrow-array", @@ -1292,7 +1319,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -1500,12 +1527,6 @@ dependencies = [ "web-sys", ] -[[package]] -name = "integer-encoding" -version = "1.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" - [[package]] name = "integer-encoding" version = "3.0.4" @@ -1990,18 +2011,9 @@ checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" [[package]] name = "oorandom" -version = "11.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" - -[[package]] -name = "ordered-float" -version = "1.1.1" +version = "11.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" -dependencies = [ - "num-traits", -] +checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9" [[package]] name = "ordered-float" @@ -2049,19 +2061,10 @@ dependencies = [ "num-bigint", "paste", "seq-macro", - "thrift 0.17.0", + "thrift", "twox-hash", ] -[[package]] -name = "parquet-format" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" -dependencies = [ - "thrift 0.13.0", -] - [[package]] name = "parse-zoneinfo" version = "0.3.1" @@ -2276,7 +2279,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2397,9 +2400,9 @@ checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" [[package]] name = "rgb" -version = "0.8.40" +version = "0.8.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7439be6844e40133eda024efd85bf07f59d0dd2f59b10c00dd6cfb92cc5c741" +checksum = "1aee83dc281d5a3200d37b299acd13b81066ea126a7f16f0eae70fc9aed241d9" dependencies = [ "bytemuck", ] @@ -2473,9 +2476,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "bc76f558e0cbb2a839d37354c575f1dc3fdc6546b5be373ba43d95f231bf7c12" dependencies = [ "serde_derive", ] @@ -2486,19 +2489,19 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float 2.10.1", + "ordered-float", "serde", ] [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "e0cd7e117be63d3c3678776753929474f3b04a43a080c744d6b0ae2a8c28e222" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2609,7 +2612,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2649,7 +2652,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2694,9 +2697,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.68" +version = "2.0.70" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "901fa70d88b9d6c98022e23b4136f9f3e54e4662c3bc1bd1d84a42a9a0f0c1e9" +checksum = "2f0209b68b3613b093e0ec905354eccaedcfe83b8cb37cbdeae64026c3064c16" dependencies = [ "proc-macro2", "quote", @@ -2717,22 +2720,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.61" +version = "1.0.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c546c80d6be4bc6a00c0f01730c08df82eaa7a7a61f11d656526506112cc1709" +checksum = "f2675633b1499176c2dff06b0856a27976a8f9d436737b4cf4f312d4d91d8bbb" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.61" +version = "1.0.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" +checksum = "d20468752b09f49e909e55a5d338caa8bedf615594e9d80bc4c565d30faf798c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2745,28 +2748,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "threadpool" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" -dependencies = [ - "num_cpus", -] - -[[package]] -name = "thrift" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" -dependencies = [ - "byteorder", - "integer-encoding 1.1.7", - "log", - "ordered-float 1.1.1", - "threadpool", -] - [[package]] name = "thrift" version = "0.17.0" @@ -2774,10 +2755,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" dependencies = [ "byteorder", - "integer-encoding 3.0.4", - "log", - "ordered-float 2.10.1", - "threadpool", + "integer-encoding", + "ordered-float", ] [[package]] @@ -2801,9 +2780,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce6b6a2fb3a985e99cebfaefa9faa3024743da73304ca1c683a36429613d3d22" +checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" dependencies = [ "tinyvec_macros", ] @@ -2835,18 +2814,7 @@ checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", -] - -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", + "syn 2.0.70", ] [[package]] @@ -2868,7 +2836,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2967,9 +2935,9 @@ dependencies = [ [[package]] name = "uuid" -version = "1.9.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5de17fd2f7da591098415cff336e12965a28061ddace43b59cb3c430179c9439" +checksum = "81dfa00651efa65069b0b6b651f4aaa31ba9e3c3ce0137aaad053604ee7e0314" dependencies = [ "getrandom", ] @@ -3017,7 +2985,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-shared", ] @@ -3039,7 +3007,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3268,7 +3236,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index 79d393fdb..4f306452a 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -16,7 +16,8 @@ # under the License. [workspace] -members = ["core"] +members = ["core", "spark-expr"] +resolver = "2" [workspace.package] version = "0.1.0" @@ -30,3 +31,31 @@ edition = "2021" # Comet uses the same minimum Rust version as DataFusion rust-version = "1.75" + +[workspace.dependencies] +arrow = { version = "52.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "52.1.0" } +arrow-buffer = { version = "52.1.0" } +arrow-data = { version = "52.1.0" } +arrow-schema = { version = "52.1.0" } +parquet = { version = "52.1.0", default-features = false, features = ["experimental"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0" } +datafusion = { default-features = false, git = "https://github.com/apache/datafusion.git", rev = "40.0.0", features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0", features = ["crypto_expressions"] } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0", default-features = false } +datafusion-physical-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "40.0.0", default-features = false } +datafusion-comet-spark-expr = { path = "spark-expr", version = "0.1.0" } +chrono = { version = "0.4", default-features = false, features = ["clock"] } +chrono-tz = { version = "0.8" } +num = "0.4" +regex = "1.9.6" +thiserror = "1" + +[profile.release] +debug = true +overflow-checks = false +lto = "thin" +codegen-units = 1 +strip = "debuginfo" diff --git a/native/core/Cargo.lock b/native/core/Cargo.lock deleted file mode 100644 index 71fe6eb2e..000000000 --- a/native/core/Cargo.lock +++ /dev/null @@ -1,3301 +0,0 @@ -# This file is automatically @generated by Cargo. -# It is not intended for manual editing. -version = 3 - -[[package]] -name = "addr2line" -version = "0.22.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e4503c46a5c0c7844e948c9a4d6acd9f50cccb4de1c48eb9e291ea17470c678" -dependencies = [ - "gimli", -] - -[[package]] -name = "adler" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" - -[[package]] -name = "ahash" -version = "0.8.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" -dependencies = [ - "cfg-if", - "const-random", - "getrandom", - "once_cell", - "version_check", - "zerocopy", -] - -[[package]] -name = "aho-corasick" -version = "1.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" -dependencies = [ - "memchr", -] - -[[package]] -name = "alloc-no-stdlib" -version = "2.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" - -[[package]] -name = "alloc-stdlib" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" -dependencies = [ - "alloc-no-stdlib", -] - -[[package]] -name = "allocator-api2" -version = "0.2.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" - -[[package]] -name = "android-tzdata" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" - -[[package]] -name = "android_system_properties" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" -dependencies = [ - "libc", -] - -[[package]] -name = "anes" -version = "0.1.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" - -[[package]] -name = "anstyle" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "038dfcf04a5feb68e9c60b21c9625a54c2c0616e79b72b0fd87075a056ae1d1b" - -[[package]] -name = "anyhow" -version = "1.0.86" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3d1d046238990b9cf5bcde22a3fb3584ee5cf65fb2765f454ed428c7a0063da" - -[[package]] -name = "arc-swap" -version = "1.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" - -[[package]] -name = "arrayref" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" - -[[package]] -name = "arrayvec" -version = "0.7.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" - -[[package]] -name = "arrow" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-arith", - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-csv", - "arrow-data", - "arrow-ipc", - "arrow-json", - "arrow-ord", - "arrow-row", - "arrow-schema", - "arrow-select", - "arrow-string", -] - -[[package]] -name = "arrow-arith" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "chrono", - "half", - "num", -] - -[[package]] -name = "arrow-array" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "chrono", - "chrono-tz 0.9.0", - "half", - "hashbrown", - "num", -] - -[[package]] -name = "arrow-buffer" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "bytes", - "half", - "num", -] - -[[package]] -name = "arrow-cast" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", - "atoi", - "base64", - "chrono", - "comfy-table", - "half", - "lexical-core", - "num", - "ryu", -] - -[[package]] -name = "arrow-csv" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-schema", - "chrono", - "csv", - "csv-core", - "lazy_static", - "lexical-core", - "regex", -] - -[[package]] -name = "arrow-data" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-buffer", - "arrow-schema", - "half", - "num", -] - -[[package]] -name = "arrow-ipc" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-schema", - "flatbuffers", - "lz4_flex", -] - -[[package]] -name = "arrow-json" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-data", - "arrow-schema", - "chrono", - "half", - "indexmap", - "lexical-core", - "num", - "serde", - "serde_json", -] - -[[package]] -name = "arrow-ord" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", - "half", - "num", -] - -[[package]] -name = "arrow-row" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "half", - "hashbrown", -] - -[[package]] -name = "arrow-schema" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "bitflags 2.5.0", -] - -[[package]] -name = "arrow-select" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "num", -] - -[[package]] -name = "arrow-string" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", - "memchr", - "num", - "regex", - "regex-syntax", -] - -[[package]] -name = "assertables" -version = "7.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" - -[[package]] -name = "async-trait" -version = "0.1.80" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "atoi" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" -dependencies = [ - "num-traits", -] - -[[package]] -name = "autocfg" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" - -[[package]] -name = "backtrace" -version = "0.3.72" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17c6a35df3749d2e8bb1b7b21a976d82b15548788d2735b9d82f329268f71a11" -dependencies = [ - "addr2line", - "cc", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", -] - -[[package]] -name = "base64" -version = "0.22.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" - -[[package]] -name = "bitflags" -version = "1.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" - -[[package]] -name = "bitflags" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1" - -[[package]] -name = "blake2" -version = "0.10.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" -dependencies = [ - "digest", -] - -[[package]] -name = "blake3" -version = "1.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30cca6d3674597c30ddf2c587bf8d9d65c9a84d2326d941cc79c9842dfe0ef52" -dependencies = [ - "arrayref", - "arrayvec", - "cc", - "cfg-if", - "constant_time_eq", -] - -[[package]] -name = "block-buffer" -version = "0.10.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" -dependencies = [ - "generic-array", -] - -[[package]] -name = "brotli" -version = "3.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", - "brotli-decompressor", -] - -[[package]] -name = "brotli-decompressor" -version = "2.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" -dependencies = [ - "alloc-no-stdlib", - "alloc-stdlib", -] - -[[package]] -name = "bumpalo" -version = "3.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" - -[[package]] -name = "bytemuck" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78834c15cb5d5efe3452d58b1e8ba890dd62d21907f867f383358198e56ebca5" - -[[package]] -name = "byteorder" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" - -[[package]] -name = "bytes" -version = "1.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" - -[[package]] -name = "cast" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" - -[[package]] -name = "cc" -version = "1.0.99" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96c51067fd44124faa7f870b4b1c969379ad32b2ba805aa959430ceaa384f695" -dependencies = [ - "jobserver", - "libc", - "once_cell", -] - -[[package]] -name = "cesu8" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" - -[[package]] -name = "cfg-if" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" - -[[package]] -name = "chrono" -version = "0.4.38" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" -dependencies = [ - "android-tzdata", - "iana-time-zone", - "num-traits", - "windows-targets 0.52.5", -] - -[[package]] -name = "chrono-tz" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d59ae0466b83e838b81a54256c39d5d7c20b9d7daa10510a242d9b75abd5936e" -dependencies = [ - "chrono", - "chrono-tz-build 0.2.1", - "phf", -] - -[[package]] -name = "chrono-tz" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93698b29de5e97ad0ae26447b344c482a7284c737d9ddc5f9e52b74a336671bb" -dependencies = [ - "chrono", - "chrono-tz-build 0.3.0", - "phf", -] - -[[package]] -name = "chrono-tz-build" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" -dependencies = [ - "parse-zoneinfo", - "phf", - "phf_codegen", -] - -[[package]] -name = "chrono-tz-build" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c088aee841df9c3041febbb73934cfc39708749bf96dc827e3359cd39ef11b1" -dependencies = [ - "parse-zoneinfo", - "phf", - "phf_codegen", -] - -[[package]] -name = "ciborium" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" -dependencies = [ - "ciborium-io", - "ciborium-ll", - "serde", -] - -[[package]] -name = "ciborium-io" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" - -[[package]] -name = "ciborium-ll" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" -dependencies = [ - "ciborium-io", - "half", -] - -[[package]] -name = "clap" -version = "4.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9689a29b593160de5bc4aacab7b5d54fb52231de70122626c178e6a368994c7" -dependencies = [ - "clap_builder", -] - -[[package]] -name = "clap_builder" -version = "4.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e5387378c84f6faa26890ebf9f0a92989f8873d4d380467bcd0d8d8620424df" -dependencies = [ - "anstyle", - "clap_lex", -] - -[[package]] -name = "clap_lex" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b82cf0babdbd58558212896d1a4272303a57bdb245c2bf1147185fb45640e70" - -[[package]] -name = "combine" -version = "4.6.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" -dependencies = [ - "bytes", - "memchr", -] - -[[package]] -name = "comfy-table" -version = "7.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" -dependencies = [ - "strum", - "strum_macros", - "unicode-width", -] - -[[package]] -name = "const-random" -version = "0.1.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" -dependencies = [ - "const-random-macro", -] - -[[package]] -name = "const-random-macro" -version = "0.1.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" -dependencies = [ - "getrandom", - "once_cell", - "tiny-keccak", -] - -[[package]] -name = "constant_time_eq" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" - -[[package]] -name = "core-foundation-sys" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" - -[[package]] -name = "cpp_demangle" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8227005286ec39567949b33df9896bcadfa6051bccca2488129f108ca23119" -dependencies = [ - "cfg-if", -] - -[[package]] -name = "cpufeatures" -version = "0.2.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" -dependencies = [ - "libc", -] - -[[package]] -name = "crc32fast" -version = "1.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" -dependencies = [ - "cfg-if", -] - -[[package]] -name = "criterion" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" -dependencies = [ - "anes", - "cast", - "ciborium", - "clap", - "criterion-plot", - "is-terminal", - "itertools 0.10.5", - "num-traits", - "once_cell", - "oorandom", - "plotters", - "rayon", - "regex", - "serde", - "serde_derive", - "serde_json", - "tinytemplate", - "walkdir", -] - -[[package]] -name = "criterion-plot" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" -dependencies = [ - "cast", - "itertools 0.10.5", -] - -[[package]] -name = "crossbeam-deque" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" -dependencies = [ - "crossbeam-epoch", - "crossbeam-utils", -] - -[[package]] -name = "crossbeam-epoch" -version = "0.9.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" -dependencies = [ - "crossbeam-utils", -] - -[[package]] -name = "crossbeam-utils" -version = "0.8.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" - -[[package]] -name = "crunchy" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" - -[[package]] -name = "crypto-common" -version = "0.1.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" -dependencies = [ - "generic-array", - "typenum", -] - -[[package]] -name = "csv" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" -dependencies = [ - "csv-core", - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "csv-core" -version = "0.1.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" -dependencies = [ - "memchr", -] - -[[package]] -name = "dashmap" -version = "5.5.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" -dependencies = [ - "cfg-if", - "hashbrown", - "lock_api", - "once_cell", - "parking_lot_core", -] - -[[package]] -name = "datafusion" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-ipc", - "arrow-schema", - "async-trait", - "bytes", - "chrono", - "dashmap", - "datafusion-common", - "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions", - "datafusion-functions-aggregate", - "datafusion-optimizer", - "datafusion-physical-expr", - "datafusion-physical-expr-common", - "datafusion-physical-plan", - "datafusion-sql", - "futures", - "glob", - "half", - "hashbrown", - "indexmap", - "itertools 0.12.1", - "log", - "num_cpus", - "object_store", - "parking_lot", - "paste", - "pin-project-lite", - "rand", - "sqlparser", - "tempfile", - "tokio", - "url", - "uuid", -] - -[[package]] -name = "datafusion-comet" -version = "0.1.0" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-string", - "assertables", - "async-trait", - "brotli", - "byteorder", - "bytes", - "chrono", - "chrono-tz 0.8.6", - "crc32fast", - "criterion", - "datafusion", - "datafusion-common", - "datafusion-expr", - "datafusion-functions", - "datafusion-physical-expr", - "datafusion-physical-expr-common", - "flate2", - "futures", - "half", - "hashbrown", - "hex", - "itertools 0.11.0", - "jni", - "lazy_static", - "log", - "log4rs", - "lz4", - "mimalloc", - "num", - "once_cell", - "parking_lot", - "parquet", - "parquet-format", - "paste", - "pprof", - "prost 0.12.6", - "prost-build", - "rand", - "regex", - "serde", - "simd-adler32", - "snap", - "tempfile", - "thiserror", - "thrift 0.17.0", - "tokio", - "tokio-stream", - "twox-hash", - "unicode-segmentation", - "zstd", -] - -[[package]] -name = "datafusion-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-schema", - "chrono", - "half", - "hashbrown", - "instant", - "libc", - "num_cpus", - "object_store", - "sqlparser", -] - -[[package]] -name = "datafusion-common-runtime" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "tokio", -] - -[[package]] -name = "datafusion-execution" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "chrono", - "dashmap", - "datafusion-common", - "datafusion-expr", - "futures", - "hashbrown", - "log", - "object_store", - "parking_lot", - "rand", - "tempfile", - "url", -] - -[[package]] -name = "datafusion-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "chrono", - "datafusion-common", - "paste", - "serde_json", - "sqlparser", - "strum", - "strum_macros", -] - -[[package]] -name = "datafusion-functions" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "base64", - "blake2", - "blake3", - "chrono", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "hashbrown", - "hex", - "itertools 0.12.1", - "log", - "md-5", - "rand", - "regex", - "sha2", - "unicode-segmentation", - "uuid", -] - -[[package]] -name = "datafusion-functions-aggregate" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-schema", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr-common", - "log", - "paste", - "sqlparser", -] - -[[package]] -name = "datafusion-optimizer" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "async-trait", - "chrono", - "datafusion-common", - "datafusion-expr", - "datafusion-physical-expr", - "hashbrown", - "indexmap", - "itertools 0.12.1", - "log", - "regex-syntax", -] - -[[package]] -name = "datafusion-physical-expr" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-ord", - "arrow-schema", - "arrow-string", - "base64", - "chrono", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate", - "datafusion-physical-expr-common", - "half", - "hashbrown", - "hex", - "indexmap", - "itertools 0.12.1", - "log", - "paste", - "petgraph", - "regex", -] - -[[package]] -name = "datafusion-physical-expr-common" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "datafusion-common", - "datafusion-expr", - "rand", -] - -[[package]] -name = "datafusion-physical-plan" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "ahash", - "arrow", - "arrow-array", - "arrow-buffer", - "arrow-ord", - "arrow-schema", - "async-trait", - "chrono", - "datafusion-common", - "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate", - "datafusion-physical-expr", - "datafusion-physical-expr-common", - "futures", - "half", - "hashbrown", - "indexmap", - "itertools 0.12.1", - "log", - "once_cell", - "parking_lot", - "pin-project-lite", - "rand", - "tokio", -] - -[[package]] -name = "datafusion-sql" -version = "39.0.0" -source = "git+https://github.com/viirya/arrow-datafusion.git?rev=17446b1#17446b1886d2872be482efa4225d2b35e5d96569" -dependencies = [ - "arrow", - "arrow-array", - "arrow-schema", - "datafusion-common", - "datafusion-expr", - "log", - "regex", - "sqlparser", - "strum", -] - -[[package]] -name = "debugid" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" -dependencies = [ - "uuid", -] - -[[package]] -name = "derivative" -version = "2.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" -dependencies = [ - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "destructure_traitobject" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c877555693c14d2f84191cfd3ad8582790fc52b5e2274b40b59cf5f5cea25c7" - -[[package]] -name = "digest" -version = "0.10.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" -dependencies = [ - "block-buffer", - "crypto-common", - "subtle", -] - -[[package]] -name = "doc-comment" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" - -[[package]] -name = "either" -version = "1.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dca9240753cf90908d7e4aac30f630662b02aebaa1b58a3cadabdb23385b58b" - -[[package]] -name = "equivalent" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" - -[[package]] -name = "errno" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" -dependencies = [ - "libc", - "windows-sys 0.52.0", -] - -[[package]] -name = "fastrand" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" - -[[package]] -name = "findshlibs" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" -dependencies = [ - "cc", - "lazy_static", - "libc", - "winapi", -] - -[[package]] -name = "fixedbitset" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" - -[[package]] -name = "flatbuffers" -version = "24.3.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" -dependencies = [ - "bitflags 1.3.2", - "rustc_version", -] - -[[package]] -name = "flate2" -version = "1.0.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" -dependencies = [ - "crc32fast", - "miniz_oxide", -] - -[[package]] -name = "fnv" -version = "1.0.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" - -[[package]] -name = "form_urlencoded" -version = "1.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" -dependencies = [ - "percent-encoding", -] - -[[package]] -name = "futures" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" -dependencies = [ - "futures-channel", - "futures-core", - "futures-executor", - "futures-io", - "futures-sink", - "futures-task", - "futures-util", -] - -[[package]] -name = "futures-channel" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" -dependencies = [ - "futures-core", - "futures-sink", -] - -[[package]] -name = "futures-core" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" - -[[package]] -name = "futures-executor" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" -dependencies = [ - "futures-core", - "futures-task", - "futures-util", -] - -[[package]] -name = "futures-io" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" - -[[package]] -name = "futures-macro" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "futures-sink" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" - -[[package]] -name = "futures-task" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" - -[[package]] -name = "futures-util" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" -dependencies = [ - "futures-channel", - "futures-core", - "futures-io", - "futures-macro", - "futures-sink", - "futures-task", - "memchr", - "pin-project-lite", - "pin-utils", - "slab", -] - -[[package]] -name = "generic-array" -version = "0.14.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" -dependencies = [ - "typenum", - "version_check", -] - -[[package]] -name = "getrandom" -version = "0.2.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" -dependencies = [ - "cfg-if", - "libc", - "wasi", -] - -[[package]] -name = "gimli" -version = "0.29.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" - -[[package]] -name = "glob" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" - -[[package]] -name = "half" -version = "2.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dd08c532ae367adf81c312a4580bc67f1d0fe8bc9c460520283f4c0ff277888" -dependencies = [ - "cfg-if", - "crunchy", - "num-traits", -] - -[[package]] -name = "hashbrown" -version = "0.14.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" -dependencies = [ - "ahash", - "allocator-api2", -] - -[[package]] -name = "heck" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" -dependencies = [ - "unicode-segmentation", -] - -[[package]] -name = "heck" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" - -[[package]] -name = "heck" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" - -[[package]] -name = "hermit-abi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" - -[[package]] -name = "hex" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" - -[[package]] -name = "home" -version = "0.5.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" -dependencies = [ - "windows-sys 0.52.0", -] - -[[package]] -name = "humantime" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" - -[[package]] -name = "iana-time-zone" -version = "0.1.60" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7ffbb5a1b541ea2561f8c41c087286cc091e21e556a4f09a8f6cbf17b69b141" -dependencies = [ - "android_system_properties", - "core-foundation-sys", - "iana-time-zone-haiku", - "js-sys", - "wasm-bindgen", - "windows-core", -] - -[[package]] -name = "iana-time-zone-haiku" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" -dependencies = [ - "cc", -] - -[[package]] -name = "idna" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" -dependencies = [ - "unicode-bidi", - "unicode-normalization", -] - -[[package]] -name = "indexmap" -version = "2.2.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" -dependencies = [ - "equivalent", - "hashbrown", -] - -[[package]] -name = "inferno" -version = "0.11.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "321f0f839cd44a4686e9504b0a62b4d69a50b62072144c71c68f5873c167b8d9" -dependencies = [ - "ahash", - "indexmap", - "is-terminal", - "itoa", - "log", - "num-format", - "once_cell", - "quick-xml", - "rgb", - "str_stack", -] - -[[package]] -name = "instant" -version = "0.1.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" -dependencies = [ - "cfg-if", - "js-sys", - "wasm-bindgen", - "web-sys", -] - -[[package]] -name = "integer-encoding" -version = "1.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" - -[[package]] -name = "integer-encoding" -version = "3.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" - -[[package]] -name = "is-terminal" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" -dependencies = [ - "hermit-abi", - "libc", - "windows-sys 0.52.0", -] - -[[package]] -name = "itertools" -version = "0.10.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" -dependencies = [ - "either", -] - -[[package]] -name = "itertools" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" -dependencies = [ - "either", -] - -[[package]] -name = "itertools" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" -dependencies = [ - "either", -] - -[[package]] -name = "itoa" -version = "1.0.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" - -[[package]] -name = "java-locator" -version = "0.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2abecabd9961c5e01405a6426687fcf1bd94a269927137e4c3cc1a7419b93fd" -dependencies = [ - "glob", - "lazy_static", -] - -[[package]] -name = "jni" -version = "0.21.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a87aa2bb7d2af34197c04845522473242e1aa17c12f4935d5856491a7fb8c97" -dependencies = [ - "cesu8", - "cfg-if", - "combine", - "java-locator", - "jni-sys", - "libloading", - "log", - "thiserror", - "walkdir", - "windows-sys 0.45.0", -] - -[[package]] -name = "jni-sys" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" - -[[package]] -name = "jobserver" -version = "0.1.31" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b099aaa34a9751c5bf0878add70444e1ed2dd73f347be99003d4577277de6e" -dependencies = [ - "libc", -] - -[[package]] -name = "js-sys" -version = "0.3.69" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" -dependencies = [ - "wasm-bindgen", -] - -[[package]] -name = "lazy_static" -version = "1.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" - -[[package]] -name = "lexical-core" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cde5de06e8d4c2faabc400238f9ae1c74d5412d03a7bd067645ccbc47070e46" -dependencies = [ - "lexical-parse-float", - "lexical-parse-integer", - "lexical-util", - "lexical-write-float", - "lexical-write-integer", -] - -[[package]] -name = "lexical-parse-float" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "683b3a5ebd0130b8fb52ba0bdc718cc56815b6a097e28ae5a6997d0ad17dc05f" -dependencies = [ - "lexical-parse-integer", - "lexical-util", - "static_assertions", -] - -[[package]] -name = "lexical-parse-integer" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d0994485ed0c312f6d965766754ea177d07f9c00c9b82a5ee62ed5b47945ee9" -dependencies = [ - "lexical-util", - "static_assertions", -] - -[[package]] -name = "lexical-util" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5255b9ff16ff898710eb9eb63cb39248ea8a5bb036bea8085b1a767ff6c4e3fc" -dependencies = [ - "static_assertions", -] - -[[package]] -name = "lexical-write-float" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accabaa1c4581f05a3923d1b4cfd124c329352288b7b9da09e766b0668116862" -dependencies = [ - "lexical-util", - "lexical-write-integer", - "static_assertions", -] - -[[package]] -name = "lexical-write-integer" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1b6f3d1f4422866b68192d62f77bc5c700bee84f3069f2469d7bc8c77852446" -dependencies = [ - "lexical-util", - "static_assertions", -] - -[[package]] -name = "libc" -version = "0.2.155" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97b3888a4aecf77e811145cadf6eef5901f4782c53886191b2f693f24761847c" - -[[package]] -name = "libloading" -version = "0.7.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b67380fd3b2fbe7527a606e18729d21c6f3951633d0500574c4dc22d2d638b9f" -dependencies = [ - "cfg-if", - "winapi", -] - -[[package]] -name = "libm" -version = "0.2.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" - -[[package]] -name = "libmimalloc-sys" -version = "0.1.38" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e7bb23d733dfcc8af652a78b7bf232f0e967710d044732185e561e47c0336b6" -dependencies = [ - "cc", - "libc", -] - -[[package]] -name = "linux-raw-sys" -version = "0.4.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" - -[[package]] -name = "lock_api" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" -dependencies = [ - "autocfg", - "scopeguard", -] - -[[package]] -name = "log" -version = "0.4.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" -dependencies = [ - "serde", -] - -[[package]] -name = "log-mdc" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a94d21414c1f4a51209ad204c1776a3d0765002c76c6abcb602a6f09f1e881c7" - -[[package]] -name = "log4rs" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0816135ae15bd0391cf284eab37e6e3ee0a6ee63d2ceeb659862bd8d0a984ca6" -dependencies = [ - "anyhow", - "arc-swap", - "chrono", - "derivative", - "fnv", - "humantime", - "libc", - "log", - "log-mdc", - "once_cell", - "parking_lot", - "rand", - "serde", - "serde-value", - "serde_json", - "serde_yaml", - "thiserror", - "thread-id", - "typemap-ors", - "winapi", -] - -[[package]] -name = "lz4" -version = "1.24.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" -dependencies = [ - "libc", - "lz4-sys", -] - -[[package]] -name = "lz4-sys" -version = "1.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" -dependencies = [ - "cc", - "libc", -] - -[[package]] -name = "lz4_flex" -version = "0.11.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" -dependencies = [ - "twox-hash", -] - -[[package]] -name = "md-5" -version = "0.10.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" -dependencies = [ - "cfg-if", - "digest", -] - -[[package]] -name = "memchr" -version = "2.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d" - -[[package]] -name = "memmap2" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe751422e4a8caa417e13c3ea66452215d7d63e19e604f4980461212f3ae1322" -dependencies = [ - "libc", -] - -[[package]] -name = "mimalloc" -version = "0.1.42" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9186d86b79b52f4a77af65604b51225e8db1d6ee7e3f41aec1e40829c71a176" -dependencies = [ - "libmimalloc-sys", -] - -[[package]] -name = "miniz_oxide" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87dfd01fe195c66b572b37921ad8803d010623c0aca821bea2302239d155cdae" -dependencies = [ - "adler", -] - -[[package]] -name = "multimap" -version = "0.8.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" - -[[package]] -name = "nix" -version = "0.26.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "598beaf3cc6fdd9a5dfb1630c2800c7acd31df7aaf0f565796fba2b53ca1af1b" -dependencies = [ - "bitflags 1.3.2", - "cfg-if", - "libc", -] - -[[package]] -name = "num" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" -dependencies = [ - "num-bigint", - "num-complex", - "num-integer", - "num-iter", - "num-rational", - "num-traits", -] - -[[package]] -name = "num-bigint" -version = "0.4.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c165a9ab64cf766f73521c0dd2cfdff64f488b8f0b3e621face3462d3db536d7" -dependencies = [ - "num-integer", - "num-traits", -] - -[[package]] -name = "num-complex" -version = "0.4.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" -dependencies = [ - "num-traits", -] - -[[package]] -name = "num-format" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" -dependencies = [ - "arrayvec", - "itoa", -] - -[[package]] -name = "num-integer" -version = "0.1.46" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" -dependencies = [ - "num-traits", -] - -[[package]] -name = "num-iter" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-rational" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" -dependencies = [ - "num-bigint", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-traits" -version = "0.2.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" -dependencies = [ - "autocfg", - "libm", -] - -[[package]] -name = "num_cpus" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" -dependencies = [ - "hermit-abi", - "libc", -] - -[[package]] -name = "object" -version = "0.35.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8ec7ab813848ba4522158d5517a6093db1ded27575b070f4177b8d12b41db5e" -dependencies = [ - "memchr", -] - -[[package]] -name = "object_store" -version = "0.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbebfd32c213ba1907fa7a9c9138015a8de2b43e30c5aa45b18f7deb46786ad6" -dependencies = [ - "async-trait", - "bytes", - "chrono", - "futures", - "humantime", - "itertools 0.12.1", - "parking_lot", - "percent-encoding", - "snafu", - "tokio", - "tracing", - "url", - "walkdir", -] - -[[package]] -name = "once_cell" -version = "1.19.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" - -[[package]] -name = "oorandom" -version = "11.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" - -[[package]] -name = "ordered-float" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" -dependencies = [ - "num-traits", -] - -[[package]] -name = "ordered-float" -version = "2.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" -dependencies = [ - "num-traits", -] - -[[package]] -name = "parking_lot" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" -dependencies = [ - "lock_api", - "parking_lot_core", -] - -[[package]] -name = "parking_lot_core" -version = "0.9.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" -dependencies = [ - "cfg-if", - "libc", - "redox_syscall", - "smallvec", - "windows-targets 0.52.5", -] - -[[package]] -name = "parquet" -version = "52.0.0" -source = "git+https://github.com/apache/arrow-rs.git?rev=0a4d8a1#0a4d8a14b58e45ef92e31541f0b51a5b25de5f10" -dependencies = [ - "ahash", - "bytes", - "chrono", - "half", - "hashbrown", - "num", - "num-bigint", - "paste", - "seq-macro", - "thrift 0.17.0", - "twox-hash", -] - -[[package]] -name = "parquet-format" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" -dependencies = [ - "thrift 0.13.0", -] - -[[package]] -name = "parse-zoneinfo" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f2a05b18d44e2957b88f96ba460715e295bc1d7510468a2f3d3b44535d26c24" -dependencies = [ - "regex", -] - -[[package]] -name = "paste" -version = "1.0.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" - -[[package]] -name = "percent-encoding" -version = "2.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" - -[[package]] -name = "petgraph" -version = "0.6.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" -dependencies = [ - "fixedbitset", - "indexmap", -] - -[[package]] -name = "phf" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" -dependencies = [ - "phf_shared", -] - -[[package]] -name = "phf_codegen" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8d39688d359e6b34654d328e262234662d16cc0f60ec8dcbe5e718709342a5a" -dependencies = [ - "phf_generator", - "phf_shared", -] - -[[package]] -name = "phf_generator" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0" -dependencies = [ - "phf_shared", - "rand", -] - -[[package]] -name = "phf_shared" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" -dependencies = [ - "siphasher", -] - -[[package]] -name = "pin-project-lite" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" - -[[package]] -name = "pin-utils" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" - -[[package]] -name = "pkg-config" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" - -[[package]] -name = "plotters" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a15b6eccb8484002195a3e44fe65a4ce8e93a625797a063735536fd59cb01cf3" -dependencies = [ - "num-traits", - "plotters-backend", - "plotters-svg", - "wasm-bindgen", - "web-sys", -] - -[[package]] -name = "plotters-backend" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "414cec62c6634ae900ea1c56128dfe87cf63e7caece0852ec76aba307cebadb7" - -[[package]] -name = "plotters-svg" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81b30686a7d9c3e010b84284bdd26a29f2138574f52f5eb6f794fc0ad924e705" -dependencies = [ - "plotters-backend", -] - -[[package]] -name = "pprof" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef5c97c51bd34c7e742402e216abdeb44d415fbe6ae41d56b114723e953711cb" -dependencies = [ - "backtrace", - "cfg-if", - "findshlibs", - "inferno", - "libc", - "log", - "nix", - "once_cell", - "parking_lot", - "smallvec", - "symbolic-demangle", - "tempfile", - "thiserror", -] - -[[package]] -name = "ppv-lite86" -version = "0.2.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" - -[[package]] -name = "proc-macro2" -version = "1.0.85" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23" -dependencies = [ - "unicode-ident", -] - -[[package]] -name = "prost" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" -dependencies = [ - "bytes", - "prost-derive 0.9.0", -] - -[[package]] -name = "prost" -version = "0.12.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "deb1435c188b76130da55f17a466d252ff7b1418b2ad3e037d127b94e3411f29" -dependencies = [ - "bytes", - "prost-derive 0.12.6", -] - -[[package]] -name = "prost-build" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" -dependencies = [ - "bytes", - "heck 0.3.3", - "itertools 0.10.5", - "lazy_static", - "log", - "multimap", - "petgraph", - "prost 0.9.0", - "prost-types", - "regex", - "tempfile", - "which", -] - -[[package]] -name = "prost-derive" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" -dependencies = [ - "anyhow", - "itertools 0.10.5", - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "prost-derive" -version = "0.12.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" -dependencies = [ - "anyhow", - "itertools 0.12.1", - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "prost-types" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" -dependencies = [ - "bytes", - "prost 0.9.0", -] - -[[package]] -name = "quick-xml" -version = "0.26.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f50b1c63b38611e7d4d7f68b82d3ad0cc71a2ad2e7f61fc10f1328d917c93cd" -dependencies = [ - "memchr", -] - -[[package]] -name = "quote" -version = "1.0.36" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" -dependencies = [ - "proc-macro2", -] - -[[package]] -name = "rand" -version = "0.8.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" -dependencies = [ - "libc", - "rand_chacha", - "rand_core", -] - -[[package]] -name = "rand_chacha" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" -dependencies = [ - "ppv-lite86", - "rand_core", -] - -[[package]] -name = "rand_core" -version = "0.6.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" -dependencies = [ - "getrandom", -] - -[[package]] -name = "rayon" -version = "1.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b418a60154510ca1a002a752ca9714984e21e4241e804d32555251faf8b78ffa" -dependencies = [ - "either", - "rayon-core", -] - -[[package]] -name = "rayon-core" -version = "1.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" -dependencies = [ - "crossbeam-deque", - "crossbeam-utils", -] - -[[package]] -name = "redox_syscall" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "469052894dcb553421e483e4209ee581a45100d31b4018de03e5a7ad86374a7e" -dependencies = [ - "bitflags 2.5.0", -] - -[[package]] -name = "regex" -version = "1.10.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c117dbdfde9c8308975b6a18d71f3f385c89461f7b3fb054288ecf2a2058ba4c" -dependencies = [ - "aho-corasick", - "memchr", - "regex-automata", - "regex-syntax", -] - -[[package]] -name = "regex-automata" -version = "0.4.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea" -dependencies = [ - "aho-corasick", - "memchr", - "regex-syntax", -] - -[[package]] -name = "regex-syntax" -version = "0.8.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adad44e29e4c806119491a7f06f03de4d1af22c3a680dd47f1e6e179439d1f56" - -[[package]] -name = "rgb" -version = "0.8.37" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05aaa8004b64fd573fc9d002f4e632d51ad4f026c2b5ba95fcb6c2f32c2c47d8" -dependencies = [ - "bytemuck", -] - -[[package]] -name = "rustc-demangle" -version = "0.1.24" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" - -[[package]] -name = "rustc_version" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" -dependencies = [ - "semver", -] - -[[package]] -name = "rustix" -version = "0.38.34" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" -dependencies = [ - "bitflags 2.5.0", - "errno", - "libc", - "linux-raw-sys", - "windows-sys 0.52.0", -] - -[[package]] -name = "rustversion" -version = "1.0.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "955d28af4278de8121b7ebeb796b6a45735dc01436d898801014aced2773a3d6" - -[[package]] -name = "ryu" -version = "1.0.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" - -[[package]] -name = "same-file" -version = "1.0.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" -dependencies = [ - "winapi-util", -] - -[[package]] -name = "scopeguard" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" - -[[package]] -name = "semver" -version = "1.0.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" - -[[package]] -name = "seq-macro" -version = "0.3.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" - -[[package]] -name = "serde" -version = "1.0.203" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" -dependencies = [ - "serde_derive", -] - -[[package]] -name = "serde-value" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" -dependencies = [ - "ordered-float 2.10.1", - "serde", -] - -[[package]] -name = "serde_derive" -version = "1.0.203" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "serde_json" -version = "1.0.117" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "455182ea6142b14f93f4bc5320a2b31c1f266b66a4a5c858b013302a5d8cbfc3" -dependencies = [ - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "serde_yaml" -version = "0.9.34+deprecated" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" -dependencies = [ - "indexmap", - "itoa", - "ryu", - "serde", - "unsafe-libyaml", -] - -[[package]] -name = "sha2" -version = "0.10.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" -dependencies = [ - "cfg-if", - "cpufeatures", - "digest", -] - -[[package]] -name = "simd-adler32" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" - -[[package]] -name = "siphasher" -version = "0.3.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" - -[[package]] -name = "slab" -version = "0.4.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" -dependencies = [ - "autocfg", -] - -[[package]] -name = "smallvec" -version = "1.13.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" - -[[package]] -name = "snafu" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" -dependencies = [ - "doc-comment", - "snafu-derive", -] - -[[package]] -name = "snafu-derive" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" -dependencies = [ - "heck 0.4.1", - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "snap" -version = "1.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" - -[[package]] -name = "sqlparser" -version = "0.47.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "295e9930cd7a97e58ca2a070541a3ca502b17f5d1fa7157376d0fabd85324f25" -dependencies = [ - "log", - "sqlparser_derive", -] - -[[package]] -name = "sqlparser_derive" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "stable_deref_trait" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" - -[[package]] -name = "static_assertions" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" - -[[package]] -name = "str_stack" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" - -[[package]] -name = "strum" -version = "0.26.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" -dependencies = [ - "strum_macros", -] - -[[package]] -name = "strum_macros" -version = "0.26.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" -dependencies = [ - "heck 0.5.0", - "proc-macro2", - "quote", - "rustversion", - "syn 2.0.66", -] - -[[package]] -name = "subtle" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" - -[[package]] -name = "symbolic-common" -version = "12.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71297dc3e250f7dbdf8adb99e235da783d690f5819fdeb4cce39d9cfb0aca9f1" -dependencies = [ - "debugid", - "memmap2", - "stable_deref_trait", - "uuid", -] - -[[package]] -name = "symbolic-demangle" -version = "12.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "424fa2c9bf2c862891b9cfd354a752751a6730fd838a4691e7f6c2c7957b9daf" -dependencies = [ - "cpp_demangle", - "rustc-demangle", - "symbolic-common", -] - -[[package]] -name = "syn" -version = "1.0.109" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "syn" -version = "2.0.66" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c42f3f41a2de00b01c0aaad383c5a45241efc8b2d1eda5661812fda5f3cdcff5" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "tempfile" -version = "3.10.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" -dependencies = [ - "cfg-if", - "fastrand", - "rustix", - "windows-sys 0.52.0", -] - -[[package]] -name = "thiserror" -version = "1.0.61" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c546c80d6be4bc6a00c0f01730c08df82eaa7a7a61f11d656526506112cc1709" -dependencies = [ - "thiserror-impl", -] - -[[package]] -name = "thiserror-impl" -version = "1.0.61" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "thread-id" -version = "4.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0ec81c46e9eb50deaa257be2f148adf052d1fb7701cfd55ccfab2525280b70b" -dependencies = [ - "libc", - "winapi", -] - -[[package]] -name = "threadpool" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" -dependencies = [ - "num_cpus", -] - -[[package]] -name = "thrift" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" -dependencies = [ - "byteorder", - "integer-encoding 1.1.7", - "log", - "ordered-float 1.1.1", - "threadpool", -] - -[[package]] -name = "thrift" -version = "0.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" -dependencies = [ - "byteorder", - "integer-encoding 3.0.4", - "log", - "ordered-float 2.10.1", - "threadpool", -] - -[[package]] -name = "tiny-keccak" -version = "2.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" -dependencies = [ - "crunchy", -] - -[[package]] -name = "tinytemplate" -version = "1.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" -dependencies = [ - "serde", - "serde_json", -] - -[[package]] -name = "tinyvec" -version = "1.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" -dependencies = [ - "tinyvec_macros", -] - -[[package]] -name = "tinyvec_macros" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" - -[[package]] -name = "tokio" -version = "1.38.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba4f4a02a7a80d6f274636f0aa95c7e383b912d41fe721a31f29e29698585a4a" -dependencies = [ - "backtrace", - "bytes", - "num_cpus", - "pin-project-lite", - "tokio-macros", -] - -[[package]] -name = "tokio-macros" -version = "2.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", -] - -[[package]] -name = "tracing" -version = "0.1.40" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" -dependencies = [ - "pin-project-lite", - "tracing-attributes", - "tracing-core", -] - -[[package]] -name = "tracing-attributes" -version = "0.1.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "tracing-core" -version = "0.1.32" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" -dependencies = [ - "once_cell", -] - -[[package]] -name = "twox-hash" -version = "1.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" -dependencies = [ - "cfg-if", - "rand", - "static_assertions", -] - -[[package]] -name = "typemap-ors" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a68c24b707f02dd18f1e4ccceb9d49f2058c2fb86384ef9972592904d7a28867" -dependencies = [ - "unsafe-any-ors", -] - -[[package]] -name = "typenum" -version = "1.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" - -[[package]] -name = "unicode-bidi" -version = "0.3.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" - -[[package]] -name = "unicode-ident" -version = "1.0.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" - -[[package]] -name = "unicode-normalization" -version = "0.1.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" -dependencies = [ - "tinyvec", -] - -[[package]] -name = "unicode-segmentation" -version = "1.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" - -[[package]] -name = "unicode-width" -version = "0.1.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" - -[[package]] -name = "unsafe-any-ors" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0a303d30665362d9680d7d91d78b23f5f899504d4f08b3c4cf08d055d87c0ad" -dependencies = [ - "destructure_traitobject", -] - -[[package]] -name = "unsafe-libyaml" -version = "0.2.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" - -[[package]] -name = "url" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" -dependencies = [ - "form_urlencoded", - "idna", - "percent-encoding", -] - -[[package]] -name = "uuid" -version = "1.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a183cf7feeba97b4dd1c0d46788634f6221d87fa961b305bed08c851829efcc0" -dependencies = [ - "getrandom", -] - -[[package]] -name = "version_check" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" - -[[package]] -name = "walkdir" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" -dependencies = [ - "same-file", - "winapi-util", -] - -[[package]] -name = "wasi" -version = "0.11.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" - -[[package]] -name = "wasm-bindgen" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" -dependencies = [ - "cfg-if", - "wasm-bindgen-macro", -] - -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" -dependencies = [ - "bumpalo", - "log", - "once_cell", - "proc-macro2", - "quote", - "syn 2.0.66", - "wasm-bindgen-shared", -] - -[[package]] -name = "wasm-bindgen-macro" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" -dependencies = [ - "quote", - "wasm-bindgen-macro-support", -] - -[[package]] -name = "wasm-bindgen-macro-support" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", - "wasm-bindgen-backend", - "wasm-bindgen-shared", -] - -[[package]] -name = "wasm-bindgen-shared" -version = "0.2.92" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" - -[[package]] -name = "web-sys" -version = "0.3.69" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" -dependencies = [ - "js-sys", - "wasm-bindgen", -] - -[[package]] -name = "which" -version = "4.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" -dependencies = [ - "either", - "home", - "once_cell", - "rustix", -] - -[[package]] -name = "winapi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" -dependencies = [ - "winapi-i686-pc-windows-gnu", - "winapi-x86_64-pc-windows-gnu", -] - -[[package]] -name = "winapi-i686-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" - -[[package]] -name = "winapi-util" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" -dependencies = [ - "windows-sys 0.52.0", -] - -[[package]] -name = "winapi-x86_64-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" - -[[package]] -name = "windows-core" -version = "0.52.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" -dependencies = [ - "windows-targets 0.52.5", -] - -[[package]] -name = "windows-sys" -version = "0.45.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" -dependencies = [ - "windows-targets 0.42.2", -] - -[[package]] -name = "windows-sys" -version = "0.52.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" -dependencies = [ - "windows-targets 0.52.5", -] - -[[package]] -name = "windows-targets" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" -dependencies = [ - "windows_aarch64_gnullvm 0.42.2", - "windows_aarch64_msvc 0.42.2", - "windows_i686_gnu 0.42.2", - "windows_i686_msvc 0.42.2", - "windows_x86_64_gnu 0.42.2", - "windows_x86_64_gnullvm 0.42.2", - "windows_x86_64_msvc 0.42.2", -] - -[[package]] -name = "windows-targets" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb" -dependencies = [ - "windows_aarch64_gnullvm 0.52.5", - "windows_aarch64_msvc 0.52.5", - "windows_i686_gnu 0.52.5", - "windows_i686_gnullvm", - "windows_i686_msvc 0.52.5", - "windows_x86_64_gnu 0.52.5", - "windows_x86_64_gnullvm 0.52.5", - "windows_x86_64_msvc 0.52.5", -] - -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" - -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263" - -[[package]] -name = "windows_aarch64_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" - -[[package]] -name = "windows_aarch64_msvc" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6" - -[[package]] -name = "windows_i686_gnu" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" - -[[package]] -name = "windows_i686_gnu" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670" - -[[package]] -name = "windows_i686_gnullvm" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9" - -[[package]] -name = "windows_i686_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" - -[[package]] -name = "windows_i686_msvc" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf" - -[[package]] -name = "windows_x86_64_gnu" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" - -[[package]] -name = "windows_x86_64_gnu" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9" - -[[package]] -name = "windows_x86_64_gnullvm" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" - -[[package]] -name = "windows_x86_64_gnullvm" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596" - -[[package]] -name = "windows_x86_64_msvc" -version = "0.42.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" - -[[package]] -name = "windows_x86_64_msvc" -version = "0.52.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0" - -[[package]] -name = "zerocopy" -version = "0.7.34" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae87e3fcd617500e5d106f0380cf7b77f3c6092aae37191433159dda23cfb087" -dependencies = [ - "zerocopy-derive", -] - -[[package]] -name = "zerocopy-derive" -version = "0.7.34" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15e934569e47891f7d9411f1a451d947a60e000ab3bd24fbb970f000387d1b3b" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - -[[package]] -name = "zstd" -version = "0.11.2+zstd.1.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" -dependencies = [ - "zstd-safe", -] - -[[package]] -name = "zstd-safe" -version = "5.0.2+zstd.1.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" -dependencies = [ - "libc", - "zstd-sys", -] - -[[package]] -name = "zstd-sys" -version = "2.0.10+zstd.1.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" -dependencies = [ - "cc", - "pkg-config", -] diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index c3e924a44..c252fad6d 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -31,57 +31,52 @@ include = [ "Cargo.toml", ] +# this crate is used in the Spark plugin and does not contain public Rust APIs so we do not publish this crate +publish = false + [dependencies] -parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate -arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } -arrow-string = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1" } +arrow = { workspace = true } +arrow-array = { workspace = true } +arrow-buffer = { workspace = true } +arrow-data = { workspace = true } +arrow-schema = { workspace = true } parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "0a4d8a1", default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } futures = "0.3.28" mimalloc = { version = "*", default-features = false, optional = true } tokio = { version = "1", features = ["rt-multi-thread"] } -tokio-stream = { version = "0.1.8" } async-trait = "0.1" log = "0.4" log4rs = "1.2.0" -thiserror = "1" +thiserror = { workspace = true } serde = { version = "1", features = ["derive"] } lazy_static = "1.4.0" prost = "0.12.1" -thrift = "0.17" jni = "0.21" -byteorder = "1.4.3" snap = "1.1" brotli = "3.3" flate2 = "1.0" lz4 = "1.24" zstd = "0.11" rand = "0.8" -num = "0.4" +num = { workspace = true } bytes = "1.5.0" -hashbrown = { version = "0.14", features = ["raw"] } -parking_lot = "0.12" tempfile = "3.8.0" ahash = { version = "0.8", default-features = false } itertools = "0.11.0" -chrono = { version = "0.4", default-features = false, features = ["clock"] } -chrono-tz = { version = "0.8" } +chrono = { workspace = true } paste = "1.0.14" -datafusion-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1" } -datafusion = { default-features = false, git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", features = ["crypto_expressions"] } -datafusion-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } -datafusion-physical-expr-common = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } -datafusion-physical-expr = { git = "https://github.com/viirya/arrow-datafusion.git", rev = "17446b1", default-features = false } +datafusion-common = { workspace = true } +datafusion = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-expr = { workspace = true } unicode-segmentation = "^1.10.1" once_cell = "1.18.0" -regex = "1.9.6" +regex = { workspace = true } crc32fast = "1.3.2" simd-adler32 = "0.3.7" +datafusion-comet-spark-expr = { workspace = true } [build-dependencies] prost-build = "0.9.0" @@ -97,13 +92,7 @@ twox-hash = "1.6.3" [features] default = [] - -[profile.release] -debug = true -overflow-checks = false -lto = "thin" -codegen-units = 1 -strip = "debuginfo" +nightly = [] [lib] name = "comet" diff --git a/native/core/src/errors.rs b/native/core/src/errors.rs index 7b0b57448..ff89e77d2 100644 --- a/native/core/src/errors.rs +++ b/native/core/src/errors.rs @@ -38,6 +38,7 @@ use std::{ use jni::sys::{jboolean, jbyte, jchar, jdouble, jfloat, jint, jlong, jobject, jshort}; use crate::execution::operators::ExecutionError; +use datafusion_comet_spark_expr::SparkError; use jni::objects::{GlobalRef, JThrowable}; use jni::JNIEnv; use lazy_static::lazy_static; @@ -62,36 +63,10 @@ pub enum CometError { #[error("Comet Internal Error: {0}")] Internal(String), - // Note that this message format is based on Spark 3.4 and is more detailed than the message - // returned by Spark 3.3 - #[error("[CAST_INVALID_INPUT] The value '{value}' of the type \"{from_type}\" cannot be cast to \"{to_type}\" \ - because it is malformed. Correct the value as per the syntax, or change its target type. \ - Use `try_cast` to tolerate malformed input and return NULL instead. If necessary \ - set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.")] - CastInvalidValue { - value: String, - from_type: String, - to_type: String, - }, - - #[error("[NUMERIC_VALUE_OUT_OF_RANGE] {value} cannot be represented as Decimal({precision}, {scale}). If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error, and return NULL instead.")] - NumericValueOutOfRange { - value: String, - precision: u8, - scale: i8, - }, - - #[error("[CAST_OVERFLOW] The value {value} of the type \"{from_type}\" cannot be cast to \"{to_type}\" \ - due to an overflow. Use `try_cast` to tolerate overflow and return NULL instead. If necessary \ - set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.")] - CastOverFlow { - value: String, - from_type: String, - to_type: String, - }, - - #[error("[ARITHMETIC_OVERFLOW] {from_type} overflow. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.")] - ArithmeticOverflow { from_type: String }, + /// CometError::Spark is typically used in native code to emulate the same errors + /// that Spark would return + #[error(transparent)] + Spark(SparkError), #[error(transparent)] Arrow { @@ -239,11 +214,7 @@ impl jni::errors::ToException for CometError { class: "java/lang/NullPointerException".to_string(), msg: self.to_string(), }, - CometError::CastInvalidValue { .. } => Exception { - class: "org/apache/spark/SparkException".to_string(), - msg: self.to_string(), - }, - CometError::NumericValueOutOfRange { .. } => Exception { + CometError::Spark { .. } => Exception { class: "org/apache/spark/SparkException".to_string(), msg: self.to_string(), }, @@ -586,6 +557,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn error_from_panic() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -604,6 +576,7 @@ mod tests { // Verify that functions that return an object are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn object_result() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -621,6 +594,7 @@ mod tests { // Verify that functions that return an native time are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_result() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -637,6 +611,7 @@ mod tests { // Verify that functions that return an array can handle throwing exceptions. The test // causes an exception by dividing by zero. #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_panic_exception() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -657,6 +632,7 @@ mod tests { // Verify that functions that return an native time are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_result_ok() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -673,6 +649,7 @@ mod tests { // Verify that functions that return an native time are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jlong_result_err() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -693,6 +670,7 @@ mod tests { // Verify that functions that return an array are handled correctly. This is basically // a test of the "happy path". #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jint_array_result() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -713,6 +691,7 @@ mod tests { // Verify that functions that return an array can handle throwing exceptions. The test // causes an exception by dividing by zero. #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn jint_array_panic_exception() { let _guard = attach_current_thread(); let mut env = jvm().get_env().unwrap(); @@ -736,6 +715,7 @@ mod tests { /// See [`object_panic_exception`] for a test which involves generating a panic and verifying /// that the resulting stack trace includes the offending call. #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `dlopen` pub fn stacktrace_string() { // Setup: Start with a backtrace that includes all of the expected scenarios, including // cases where the file and location are not provided as part of the backtrace capture diff --git a/native/core/src/execution/datafusion/expressions/avg.rs b/native/core/src/execution/datafusion/expressions/avg.rs index 1ff276e5d..3c8865bd1 100644 --- a/native/core/src/execution/datafusion/expressions/avg.rs +++ b/native/core/src/execution/datafusion/expressions/avg.rs @@ -47,7 +47,7 @@ pub struct Avg { impl Avg { /// Create a new AVG aggregate function pub fn new(expr: Arc, name: impl Into, data_type: DataType) -> Self { - let result_data_type = avg_return_type(&data_type).unwrap(); + let result_data_type = avg_return_type("avg", &data_type).unwrap(); Self { name: name.into(), diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 385857f1e..f6fb26b6a 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -18,16 +18,13 @@ //! Native DataFusion expressions pub mod bitwise_not; -pub mod cast; +pub use datafusion_comet_spark_expr::cast; pub mod checkoverflow; -pub mod if_expr; mod normalize_nan; pub mod scalar_funcs; pub use normalize_nan::NormalizeNaNAndZero; -use prost::DecodeError; -use crate::{errors::CometError, execution::spark_expression}; -pub mod abs; +use crate::errors::CometError; pub mod avg; pub mod avg_decimal; pub mod bloom_filter_might_contain; @@ -46,27 +43,10 @@ mod utils; pub mod variance; pub mod xxhash64; -#[derive(Debug, Hash, PartialEq, Clone, Copy)] -pub enum EvalMode { - Legacy, - Ansi, - Try, -} - -impl TryFrom for EvalMode { - type Error = DecodeError; - - fn try_from(value: i32) -> Result { - match spark_expression::EvalMode::try_from(value)? { - spark_expression::EvalMode::Legacy => Ok(EvalMode::Legacy), - spark_expression::EvalMode::Try => Ok(EvalMode::Try), - spark_expression::EvalMode::Ansi => Ok(EvalMode::Ansi), - } - } -} +pub use datafusion_comet_spark_expr::{EvalMode, SparkError}; fn arithmetic_overflow_error(from_type: &str) -> CometError { - CometError::ArithmeticOverflow { + CometError::Spark(SparkError::ArithmeticOverflow { from_type: from_type.to_string(), - } + }) } diff --git a/native/core/src/execution/datafusion/expressions/negative.rs b/native/core/src/execution/datafusion/expressions/negative.rs index cd0e9bccf..9e82812be 100644 --- a/native/core/src/execution/datafusion/expressions/negative.rs +++ b/native/core/src/execution/datafusion/expressions/negative.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use super::arithmetic_overflow_error; use crate::errors::CometError; use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeType}; use arrow_array::RecordBatch; @@ -24,6 +25,7 @@ use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, }; +use datafusion_comet_spark_expr::SparkError; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; use datafusion_physical_expr::aggregate::utils::down_cast_any_ref; @@ -33,8 +35,6 @@ use std::{ sync::Arc, }; -use super::arithmetic_overflow_error; - pub fn create_negate_expr( expr: Arc, fail_on_error: bool, @@ -234,7 +234,7 @@ impl PhysicalExpr for NegativeExpr { || child_interval.lower() == &ScalarValue::Int64(Some(i64::MIN)) || child_interval.upper() == &ScalarValue::Int64(Some(i64::MIN)) { - return Err(CometError::ArithmeticOverflow { + return Err(SparkError::ArithmeticOverflow { from_type: "long".to_string(), } .into()); diff --git a/native/core/src/execution/datafusion/expressions/utils.rs b/native/core/src/execution/datafusion/expressions/utils.rs index ee8646a78..d253b251f 100644 --- a/native/core/src/execution/datafusion/expressions/utils.rs +++ b/native/core/src/execution/datafusion/expressions/utils.rs @@ -15,264 +15,5 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::timezone::Tz; -use arrow::{ - array::{ - as_dictionary_array, as_primitive_array, Array, ArrayRef, GenericStringArray, - PrimitiveArray, - }, - compute::unary, - datatypes::{Int32Type, Int64Type, TimestampMicrosecondType}, - error::ArrowError, - temporal_conversions::as_datetime, -}; -use arrow_array::{cast::AsArray, types::ArrowPrimitiveType}; -use arrow_schema::DataType; -use chrono::{DateTime, Offset, TimeZone}; -use datafusion_common::cast::as_generic_string_array; -use datafusion_physical_expr::PhysicalExpr; -use num::integer::div_floor; -use std::{any::Any, sync::Arc}; - -/// An utility function from DataFusion. It is not exposed by DataFusion. -pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { - if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else { - any - } -} - -/// Preprocesses input arrays to add timezone information from Spark to Arrow array datatype or -/// to apply timezone offset. -// -// We consider the following cases: -// -// | --------------------- | ------------ | ----------------- | -------------------------------- | -// | Conversion | Input array | Timezone | Output array | -// | --------------------- | ------------ | ----------------- | -------------------------------- | -// | Timestamp -> | Array in UTC | Timezone of input | A timestamp with the timezone | -// | Utf8 or Date32 | | | offset applied and timezone | -// | | | | removed | -// | --------------------- | ------------ | ----------------- | -------------------------------- | -// | Timestamp -> | Array in UTC | Timezone of input | Same as input array | -// | Timestamp w/Timezone| | | | -// | --------------------- | ------------ | ----------------- | -------------------------------- | -// | Timestamp_ntz -> | Array in | Timezone of input | Same as input array | -// | Utf8 or Date32 | timezone | | | -// | | session local| | | -// | | timezone | | | -// | --------------------- | ------------ | ----------------- | -------------------------------- | -// | Timestamp_ntz -> | Array in | Timezone of input | Array in UTC and timezone | -// | Timestamp w/Timezone | session local| | specified in input | -// | | timezone | | | -// | --------------------- | ------------ | ----------------- | -------------------------------- | -// | Timestamp(_ntz) -> | | -// | Any other type | Not Supported | -// | --------------------- | ------------ | ----------------- | -------------------------------- | -// -pub fn array_with_timezone( - array: ArrayRef, - timezone: String, - to_type: Option<&DataType>, -) -> Result { - match array.data_type() { - DataType::Timestamp(_, None) => { - assert!(!timezone.is_empty()); - match to_type { - Some(DataType::Utf8) | Some(DataType::Date32) => Ok(array), - Some(DataType::Timestamp(_, Some(_))) => { - timestamp_ntz_to_timestamp(array, timezone.as_str(), Some(timezone.as_str())) - } - _ => { - // Not supported - panic!( - "Cannot convert from {:?} to {:?}", - array.data_type(), - to_type.unwrap() - ) - } - } - } - DataType::Timestamp(_, Some(_)) => { - assert!(!timezone.is_empty()); - let array = as_primitive_array::(&array); - let array_with_timezone = array.clone().with_timezone(timezone.clone()); - let array = Arc::new(array_with_timezone) as ArrayRef; - match to_type { - Some(DataType::Utf8) | Some(DataType::Date32) => { - pre_timestamp_cast(array, timezone) - } - _ => Ok(array), - } - } - DataType::Dictionary(_, value_type) - if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => - { - let dict = as_dictionary_array::(&array); - let array = as_primitive_array::(dict.values()); - let array_with_timezone = - array_with_timezone(Arc::new(array.clone()) as ArrayRef, timezone, to_type)?; - let dict = dict.with_values(array_with_timezone); - Ok(Arc::new(dict)) - } - _ => Ok(array), - } -} - -fn datetime_cast_err(value: i64) -> ArrowError { - ArrowError::CastError(format!( - "Cannot convert TimestampMicrosecondType {value} to datetime. Comet only supports dates between Jan 1, 262145 BCE and Dec 31, 262143 CE", - )) -} - -/// Takes in a Timestamp(Microsecond, None) array and a timezone id, and returns -/// a Timestamp(Microsecond, Some<_>) array. -/// The understanding is that the input array has time in the timezone specified in the second -/// argument. -/// Parameters: -/// array - input array of timestamp without timezone -/// tz - timezone of the values in the input array -/// to_timezone - timezone to change the input values to -fn timestamp_ntz_to_timestamp( - array: ArrayRef, - tz: &str, - to_timezone: Option<&str>, -) -> Result { - assert!(!tz.is_empty()); - match array.data_type() { - DataType::Timestamp(_, None) => { - let array = as_primitive_array::(&array); - let tz: Tz = tz.parse()?; - let array: PrimitiveArray = array.try_unary(|value| { - as_datetime::(value) - .ok_or_else(|| datetime_cast_err(value)) - .map(|local_datetime| { - let datetime: DateTime = - tz.from_local_datetime(&local_datetime).unwrap(); - datetime.timestamp_micros() - }) - })?; - let array_with_tz = if let Some(to_tz) = to_timezone { - array.with_timezone(to_tz) - } else { - array - }; - Ok(Arc::new(array_with_tz)) - } - _ => Ok(array), - } -} - -const MICROS_PER_SECOND: i64 = 1000000; - -/// This takes for special pre-casting cases of Spark. E.g., Timestamp to String. -fn pre_timestamp_cast(array: ArrayRef, timezone: String) -> Result { - assert!(!timezone.is_empty()); - match array.data_type() { - DataType::Timestamp(_, _) => { - // Spark doesn't output timezone while casting timestamp to string, but arrow's cast - // kernel does if timezone exists. So we need to apply offset of timezone to array - // timestamp value and remove timezone from array datatype. - let array = as_primitive_array::(&array); - - let tz: Tz = timezone.parse()?; - let array: PrimitiveArray = array.try_unary(|value| { - as_datetime::(value) - .ok_or_else(|| datetime_cast_err(value)) - .map(|datetime| { - let offset = tz.offset_from_utc_datetime(&datetime).fix(); - let datetime = datetime + offset; - datetime.and_utc().timestamp_micros() - }) - })?; - - Ok(Arc::new(array)) - } - _ => Ok(array), - } -} - -/// This takes for special casting cases of Spark. E.g., Timestamp to Long. -/// This function runs as a post process of the DataFusion cast(). By the time it arrives here, -/// Dictionary arrays are already unpacked by the DataFusion cast() since Spark cannot specify -/// Dictionary as to_type. The from_type is taken before the DataFusion cast() runs in -/// expressions/cast.rs, so it can be still Dictionary. -pub(crate) fn spark_cast(array: ArrayRef, from_type: &DataType, to_type: &DataType) -> ArrayRef { - match (from_type, to_type) { - (DataType::Timestamp(_, _), DataType::Int64) => { - // See Spark's `Cast` expression - unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap() - } - (DataType::Dictionary(_, value_type), DataType::Int64) - if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => - { - // See Spark's `Cast` expression - unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap() - } - (DataType::Timestamp(_, _), DataType::Utf8) => remove_trailing_zeroes(array), - (DataType::Dictionary(_, value_type), DataType::Utf8) - if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => - { - remove_trailing_zeroes(array) - } - _ => array, - } -} - -/// A fork & modified version of Arrow's `unary_dyn` which is being deprecated -fn unary_dyn(array: &ArrayRef, op: F) -> Result -where - T: ArrowPrimitiveType, - F: Fn(T::Native) -> T::Native, -{ - if let Some(d) = array.as_any_dictionary_opt() { - let new_values = unary_dyn::(d.values(), op)?; - return Ok(Arc::new(d.with_values(Arc::new(new_values)))); - } - - match array.as_primitive_opt::() { - Some(a) if PrimitiveArray::::is_compatible(a.data_type()) => { - Ok(Arc::new(unary::( - array.as_any().downcast_ref::>().unwrap(), - op, - ))) - } - _ => Err(ArrowError::NotYetImplemented(format!( - "Cannot perform unary operation of type {} on array of type {}", - T::DATA_TYPE, - array.data_type() - ))), - } -} - -/// Remove any trailing zeroes in the string if they occur after in the fractional seconds, -/// to match Spark behavior -/// example: -/// "1970-01-01 05:29:59.900" => "1970-01-01 05:29:59.9" -/// "1970-01-01 05:29:59.990" => "1970-01-01 05:29:59.99" -/// "1970-01-01 05:29:59.999" => "1970-01-01 05:29:59.999" -/// "1970-01-01 05:30:00" => "1970-01-01 05:30:00" -/// "1970-01-01 05:30:00.001" => "1970-01-01 05:30:00.001" -fn remove_trailing_zeroes(array: ArrayRef) -> ArrayRef { - let string_array = as_generic_string_array::(&array).unwrap(); - let result = string_array - .iter() - .map(|s| s.map(trim_end)) - .collect::>(); - Arc::new(result) as ArrayRef -} - -fn trim_end(s: &str) -> &str { - if s.rfind('.').is_some() { - s.trim_end_matches('0') - } else { - s - } -} +// re-export for legacy reasons +pub use datafusion_comet_spark_expr::utils::{array_with_timezone, down_cast_any_ref}; diff --git a/native/core/src/execution/datafusion/expressions/xxhash64.rs b/native/core/src/execution/datafusion/expressions/xxhash64.rs index 94b9e04ba..508cfe59b 100644 --- a/native/core/src/execution/datafusion/expressions/xxhash64.rs +++ b/native/core/src/execution/datafusion/expressions/xxhash64.rs @@ -162,6 +162,7 @@ mod test { use twox_hash::XxHash64; #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn test_xxhash64_random() { let mut rng = rand::thread_rng(); for len in 0..128 { diff --git a/native/core/src/execution/datafusion/operators/expand.rs b/native/core/src/execution/datafusion/operators/expand.rs index 5285dfb46..67171212f 100644 --- a/native/core/src/execution/datafusion/operators/expand.rs +++ b/native/core/src/execution/datafusion/operators/expand.rs @@ -126,6 +126,10 @@ impl ExecutionPlan for CometExpandExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "CometExpandExec" + } } pub struct ExpandStream { diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 40515c0c4..23960c307 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -20,19 +20,22 @@ use std::{collections::HashMap, sync::Arc}; use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use datafusion::functions_aggregate::bit_and_or_xor::{bit_and_udaf, bit_or_udaf, bit_xor_udaf}; +use datafusion::functions_aggregate::count::count_udaf; +use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::physical_plan::windows::BoundedWindowAggExec; use datafusion::physical_plan::InputOrderMode; use datafusion::{ arrow::{compute::SortOptions, datatypes::SchemaRef}, common::DataFusionError, execution::FunctionRegistry, + functions_aggregate::first_last::{FirstValue, LastValue}, logical_expr::Operator as DataFusionOperator, physical_expr::{ execution_props::ExecutionProps, expressions::{ - in_list, BinaryExpr, BitAnd, BitOr, BitXor, CaseExpr, CastExpr, Column, Count, - FirstValue, IsNotNullExpr, IsNullExpr, LastValue, Literal as DataFusionLiteral, Max, - Min, NotExpr, Sum, + in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, + Literal as DataFusionLiteral, Max, Min, NotExpr, }, AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }, @@ -76,7 +79,6 @@ use crate::{ checkoverflow::CheckOverflow, correlation::Correlation, covariance::Covariance, - if_expr::IfExpr, negative, scalar_funcs::create_comet_physical_fun, stats::StatsType, @@ -104,7 +106,8 @@ use crate::{ }, }; -use super::expressions::{abs::CometAbsFunc, create_named_struct::CreateNamedStruct, EvalMode}; +use super::expressions::{create_named_struct::CreateNamedStruct, EvalMode}; +use datafusion_comet_spark_expr::{Abs, IfExpr}; // For clippy error on type_complexity. type ExecResult = Result; @@ -365,7 +368,7 @@ impl PhysicalPlanner { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); let timezone = expr.timezone.clone(); - let eval_mode = expr.eval_mode.try_into()?; + let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; Ok(Arc::new(Cast::new(child, datatype, eval_mode, timezone))) } @@ -504,8 +507,8 @@ impl PhysicalPlanner { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?; let return_type = child.data_type(&input_schema)?; let args = vec![child]; - let eval_mode = expr.eval_mode.try_into()?; - let comet_abs = Arc::new(ScalarUDF::new_from_impl(CometAbsFunc::new( + let eval_mode = from_protobuf_eval_mode(expr.eval_mode)?; + let comet_abs = Arc::new(ScalarUDF::new_from_impl(Abs::new( eval_mode, return_type.to_string(), )?)); @@ -647,7 +650,7 @@ impl PhysicalPlanner { let left = self.create_expr(left, input_schema.clone())?; let right = self.create_expr(right, input_schema.clone())?; match ( - op, + &op, left.data_type(&input_schema), right.data_type(&input_schema), ) { @@ -1208,11 +1211,19 @@ impl PhysicalPlanner { .iter() .map(|child| self.create_expr(child, schema.clone())) .collect::, _>>()?; - Ok(Arc::new(Count::new_with_multiple_exprs( - children, + + create_aggregate_expr( + &count_udaf(), + &children, + &[], + &[], + &[], + schema.as_ref(), "count", - DataType::Int64, - ))) + false, + false, + ) + .map_err(|e| ExecutionError::DataFusionError(e.to_string())) } AggExprStruct::Min(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; @@ -1236,7 +1247,18 @@ impl PhysicalPlanner { // cast to the result data type of SUM if necessary, we should not expect // a cast failure since it should have already been checked at Spark side let child = Arc::new(CastExpr::new(child, datatype.clone(), None)); - Ok(Arc::new(Sum::new(child, "sum", datatype))) + create_aggregate_expr( + &sum_udaf(), + &[child], + &[], + &[], + &[], + schema.as_ref(), + "sum", + false, + false, + ) + .map_err(|e| ExecutionError::DataFusionError(e.to_string())) } } } @@ -1263,31 +1285,79 @@ impl PhysicalPlanner { AggExprStruct::First(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - - create_aggregate_expr(&func, &[child], &[], &[], &schema, "first", false, false) - .map_err(|e| e.into()) + create_aggregate_expr( + &func, + &[child], + &[], + &[], + &[], + &schema, + "first", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::Last(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); - - create_aggregate_expr(&func, &[child], &[], &[], &schema, "last", false, false) - .map_err(|e| e.into()) + create_aggregate_expr( + &func, + &[child], + &[], + &[], + &[], + &schema, + "last", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitAndAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitAnd::new(child, "bit_and", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_and_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_and", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitOrAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitOr::new(child, "bit_or", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_or_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_or", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::BitXorAgg(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; - let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); - Ok(Arc::new(BitXor::new(child, "bit_xor", datatype))) + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + create_aggregate_expr( + &bit_xor_udaf(), + &[child], + &[], + &[], + &[], + &schema, + "bit_xor", + false, + false, + ) + .map_err(|e| e.into()) } AggExprStruct::Covariance(expr) => { let child1 = self.create_expr(expr.child1.as_ref().unwrap(), schema.clone())?; @@ -1483,6 +1553,7 @@ impl PhysicalPlanner { &window_func, window_func_name, &window_args, + &[], partition_by, sort_exprs, window_frame.into(), @@ -1743,6 +1814,14 @@ fn rewrite_physical_expr( Ok(expr.rewrite(&mut rewriter).data()?) } +fn from_protobuf_eval_mode(value: i32) -> Result { + match spark_expression::EvalMode::try_from(value)? { + spark_expression::EvalMode::Legacy => Ok(EvalMode::Legacy), + spark_expression::EvalMode::Try => Ok(EvalMode::Try), + spark_expression::EvalMode::Ansi => Ok(EvalMode::Ansi), + } +} + #[cfg(test)] mod tests { use std::{sync::Arc, task::Poll}; diff --git a/native/core/src/execution/datafusion/shuffle_writer.rs b/native/core/src/execution/datafusion/shuffle_writer.rs index 5afc9a53e..3b934813d 100644 --- a/native/core/src/execution/datafusion/shuffle_writer.rs +++ b/native/core/src/execution/datafusion/shuffle_writer.rs @@ -160,6 +160,10 @@ impl ExecutionPlan for ShuffleWriterExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "ShuffleWriterExec" + } } impl ShuffleWriterExec { @@ -1447,6 +1451,7 @@ mod test { } #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx` fn test_insert_larger_batch() { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)])); let mut b = StringBuilder::new(); diff --git a/native/core/src/execution/datafusion/spark_hash.rs b/native/core/src/execution/datafusion/spark_hash.rs index 1f0658aec..15bded3a8 100644 --- a/native/core/src/execution/datafusion/spark_hash.rs +++ b/native/core/src/execution/datafusion/spark_hash.rs @@ -89,10 +89,7 @@ pub(crate) fn spark_compatible_murmur3_hash>(data: T, seed: u32) // data is &[u8] so we do not need to check for proper alignment unsafe { let mut h1 = if len_aligned > 0 { - hash_bytes_by_int( - std::slice::from_raw_parts(data.get_unchecked(0), len_aligned), - seed, - ) + hash_bytes_by_int(&data[0..len_aligned], seed) } else { seed as i32 }; diff --git a/native/core/src/execution/kernels/temporal.rs b/native/core/src/execution/kernels/temporal.rs index 1868c6fe5..9cf35af1a 100644 --- a/native/core/src/execution/kernels/temporal.rs +++ b/native/core/src/execution/kernels/temporal.rs @@ -824,6 +824,7 @@ mod tests { use std::sync::Arc; #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn test_date_trunc() { let size = 1000; let mut vec: Vec = Vec::with_capacity(size); @@ -962,6 +963,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri fn test_timestamp_trunc() { let size = 1000; let mut vec: Vec = Vec::with_capacity(size); @@ -998,6 +1000,8 @@ mod tests { } #[test] + // test takes too long with miri + #[cfg_attr(miri, ignore)] // This test only verifies that the various input array types work. Actually correctness to // ensure this produces the same results as spark is verified in the JVM tests fn test_timestamp_trunc_array_fmt_dyn() { diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index b3be83b5f..cdd429231 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -26,7 +26,7 @@ pub mod operators; pub mod serde; pub mod shuffle; pub(crate) mod sort; -mod timezone; +pub use datafusion_comet_spark_expr::timezone; pub(crate) mod utils; mod memory_pool; diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index d011b3cb2..68c91aafc 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -126,6 +126,10 @@ impl ExecutionPlan for CopyExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "CopyExec" + } } struct CopyStream { diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index de5328210..68dd773cf 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -270,6 +270,10 @@ impl ExecutionPlan for ScanExec { fn properties(&self) -> &PlanProperties { &self.cache } + + fn name(&self) -> &str { + "ScanExec" + } } impl DisplayAs for ScanExec { diff --git a/native/core/src/execution/sort.rs b/native/core/src/execution/sort.rs index eeeb11d5b..b8687652c 100644 --- a/native/core/src/execution/sort.rs +++ b/native/core/src/execution/sort.rs @@ -165,7 +165,7 @@ where // because they are defined as Vec> ptr::copy_nonoverlapping( bucket.as_ptr(), - self.get_unchecked_mut(pos), + self.as_mut_ptr().add(pos), bucket.len(), ); } diff --git a/native/core/rustfmt.toml b/native/rustfmt.toml similarity index 100% rename from native/core/rustfmt.toml rename to native/rustfmt.toml diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml new file mode 100644 index 000000000..976a1f36f --- /dev/null +++ b/native/spark-expr/Cargo.toml @@ -0,0 +1,47 @@ +# 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] +name = "datafusion-comet-spark-expr" +description = "DataFusion expressions that emulate Apache Spark's behavior" +version = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +authors = { workspace = true } +readme = { workspace = true } +license = { workspace = true } +edition = { workspace = true } + +[dependencies] +arrow = { workspace = true } +arrow-array = { workspace = true } +arrow-schema = { workspace = true } +chrono = { workspace = true } +datafusion = { workspace = true } +datafusion-common = { workspace = true } +datafusion-functions = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-plan = { workspace = true } +chrono-tz = { workspace = true } +num = { workspace = true } +regex = { workspace = true } +thiserror = { workspace = true } + +[lib] +name = "datafusion_comet_spark_expr" +path = "src/lib.rs" diff --git a/native/spark-expr/README.md b/native/spark-expr/README.md new file mode 100644 index 000000000..a7ee75363 --- /dev/null +++ b/native/spark-expr/README.md @@ -0,0 +1,23 @@ + + +# datafusion-comet-spark-expr: Spark-compatible Expressions + +This crate provides Apache Spark-compatible expressions for use with DataFusion and is maintained as part of the +[Apache DataFusion Comet](https://github.com/apache/datafusion-comet/) subproject. \ No newline at end of file diff --git a/native/core/src/execution/datafusion/expressions/abs.rs b/native/spark-expr/src/abs.rs similarity index 79% rename from native/core/src/execution/datafusion/expressions/abs.rs rename to native/spark-expr/src/abs.rs index 4eb8c7c1e..fa25a7775 100644 --- a/native/core/src/execution/datafusion/expressions/abs.rs +++ b/native/spark-expr/src/abs.rs @@ -15,34 +15,37 @@ // specific language governing permissions and limitations // under the License. +//! Spark-compatible implementation of abs function + +use std::{any::Any, sync::Arc}; + use arrow::datatypes::DataType; use arrow_schema::ArrowError; + use datafusion::logical_expr::{ColumnarValue, ScalarUDFImpl, Signature}; use datafusion_common::DataFusionError; use datafusion_functions::math; -use std::{any::Any, sync::Arc}; - -use crate::execution::operators::ExecutionError; -use super::{arithmetic_overflow_error, EvalMode}; +use super::{EvalMode, SparkError}; +/// Spark-compatible ABS expression #[derive(Debug)] -pub struct CometAbsFunc { +pub struct Abs { inner_abs_func: Arc, eval_mode: EvalMode, data_type_name: String, } -impl CometAbsFunc { - pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result { +impl Abs { + pub fn new(eval_mode: EvalMode, data_type_name: String) -> Result { if let EvalMode::Legacy | EvalMode::Ansi = eval_mode { Ok(Self { - inner_abs_func: math::abs().inner(), + inner_abs_func: math::abs().inner().clone(), eval_mode, data_type_name, }) } else { - Err(ExecutionError::GeneralError(format!( + Err(DataFusionError::Execution(format!( "Invalid EvalMode: \"{:?}\"", eval_mode ))) @@ -50,7 +53,7 @@ impl CometAbsFunc { } } -impl ScalarUDFImpl for CometAbsFunc { +impl ScalarUDFImpl for Abs { fn as_any(&self) -> &dyn Any { self } @@ -68,17 +71,16 @@ impl ScalarUDFImpl for CometAbsFunc { fn invoke(&self, args: &[ColumnarValue]) -> Result { match self.inner_abs_func.invoke(args) { - Err(DataFusionError::ArrowError(ArrowError::ComputeError(msg), trace)) + Err(DataFusionError::ArrowError(ArrowError::ComputeError(msg), _)) if msg.contains("overflow") => { if self.eval_mode == EvalMode::Legacy { Ok(args[0].clone()) } else { - let msg = arithmetic_overflow_error(&self.data_type_name).to_string(); - Err(DataFusionError::ArrowError( - ArrowError::ComputeError(msg), - trace, - )) + Err(SparkError::ArithmeticOverflow { + from_type: self.data_type_name.clone(), + } + .into()) } } other => other, diff --git a/native/core/src/execution/datafusion/expressions/cast.rs b/native/spark-expr/src/cast.rs similarity index 90% rename from native/core/src/execution/datafusion/expressions/cast.rs rename to native/spark-expr/src/cast.rs index cdd34c19c..8702ce707 100644 --- a/native/core/src/execution/datafusion/expressions/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -24,39 +24,46 @@ use std::{ }; use arrow::{ - compute::{cast_with_options, take, CastOptions}, + array::{ + cast::AsArray, + types::{Date32Type, Int16Type, Int32Type, Int8Type}, + Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Float64Array, + GenericStringArray, Int16Array, Int32Array, Int64Array, Int8Array, OffsetSizeTrait, + PrimitiveArray, + }, + compute::{cast_with_options, take, unary, CastOptions}, datatypes::{ - ArrowPrimitiveType, Decimal128Type, DecimalType, Float32Type, Float64Type, + ArrowPrimitiveType, Decimal128Type, DecimalType, Float32Type, Float64Type, Int64Type, TimestampMicrosecondType, }, + error::ArrowError, record_batch::RecordBatch, util::display::FormatOptions, }; -use arrow_array::{ - types::{Date32Type, Int16Type, Int32Type, Int64Type, Int8Type}, - Array, ArrayRef, BooleanArray, Decimal128Array, DictionaryArray, Float32Array, Float64Array, - GenericStringArray, Int16Array, Int32Array, Int64Array, Int8Array, OffsetSizeTrait, - PrimitiveArray, -}; +use arrow_array::DictionaryArray; use arrow_schema::{DataType, Schema}; -use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::logical_expr::ColumnarValue; -use datafusion_common::{internal_err, Result as DataFusionResult, ScalarValue}; + +use datafusion_common::{ + cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, +}; +use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; -use num::{cast::AsPrimitive, traits::CheckedNeg, CheckedSub, Integer, Num, ToPrimitive}; -use regex::Regex; -use crate::{ - errors::{CometError, CometResult}, - execution::datafusion::expressions::utils::{ - array_with_timezone, down_cast_any_ref, spark_cast, - }, +use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; +use num::{ + cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, + ToPrimitive, }; +use regex::Regex; -use super::EvalMode; +use crate::utils::{array_with_timezone, down_cast_any_ref}; + +use crate::{EvalMode, SparkError, SparkResult}; static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f"); +const MICROS_PER_SECOND: i64 = 1000000; + static CAST_OPTIONS: CastOptions = CastOptions { safe: true, format_options: FormatOptions::new() @@ -88,7 +95,7 @@ macro_rules! cast_utf8_to_int { cast_array.append_null() } } - let result: CometResult = Ok(Arc::new(cast_array.finish()) as ArrayRef); + let result: SparkResult = Ok(Arc::new(cast_array.finish()) as ArrayRef); result }}; } @@ -116,7 +123,7 @@ macro_rules! cast_float_to_string { fn cast( from: &dyn Array, _eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { let array = from.as_any().downcast_ref::<$output_type>().unwrap(); @@ -169,7 +176,7 @@ macro_rules! cast_float_to_string { Some(value) => Ok(Some(value.to_string())), _ => Ok(None), }) - .collect::, CometError>>()?; + .collect::, SparkError>>()?; Ok(Arc::new(output_array)) } @@ -205,7 +212,7 @@ macro_rules! cast_int_to_int_macro { .iter() .map(|value| match value { Some(value) => { - Ok::, CometError>(Some(value as $to_native_type)) + Ok::, SparkError>(Some(value as $to_native_type)) } _ => Ok(None), }) @@ -222,14 +229,14 @@ macro_rules! cast_int_to_int_macro { $spark_to_data_type_name, )) } else { - Ok::, CometError>(Some(res.unwrap())) + Ok::, SparkError>(Some(res.unwrap())) } } _ => Ok(None), }) .collect::, _>>(), }?; - let result: CometResult = Ok(Arc::new(output_array) as ArrayRef); + let result: SparkResult = Ok(Arc::new(output_array) as ArrayRef); result }}; } @@ -286,7 +293,7 @@ macro_rules! cast_float_to_int16_down { .map(|value| match value { Some(value) => { let i32_value = value as i32; - Ok::, CometError>(Some( + Ok::, SparkError>(Some( i32_value as $rust_dest_type, )) } @@ -339,7 +346,7 @@ macro_rules! cast_float_to_int32_up { .iter() .map(|value| match value { Some(value) => { - Ok::, CometError>(Some(value as $rust_dest_type)) + Ok::, SparkError>(Some(value as $rust_dest_type)) } None => Ok(None), }) @@ -402,7 +409,7 @@ macro_rules! cast_decimal_to_int16_down { Some(value) => { let divisor = 10_i128.pow($scale as u32); let i32_value = (value / divisor) as i32; - Ok::, CometError>(Some( + Ok::, SparkError>(Some( i32_value as $rust_dest_type, )) } @@ -456,7 +463,7 @@ macro_rules! cast_decimal_to_int32_up { Some(value) => { let divisor = 10_i128.pow($scale as u32); let truncated = value / divisor; - Ok::, CometError>(Some( + Ok::, SparkError>(Some( truncated as $rust_dest_type, )) } @@ -604,7 +611,7 @@ impl Cast { // we should never reach this code because the Scala code should be checking // for supported cast operations and falling back to Spark for anything that // is not yet supported - Err(CometError::Internal(format!( + Err(SparkError::Internal(format!( "Native cast invoked for unsupported cast from {from_type:?} to {to_type:?}" ))) } @@ -688,7 +695,7 @@ impl Cast { to_type: &DataType, array: &ArrayRef, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { let string_array = array .as_any() .downcast_ref::>() @@ -719,7 +726,7 @@ impl Cast { array: &ArrayRef, to_type: &DataType, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { let string_array = array .as_any() .downcast_ref::>() @@ -756,7 +763,7 @@ impl Cast { array: &ArrayRef, to_type: &DataType, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { let string_array = array .as_any() .downcast_ref::>() @@ -781,7 +788,7 @@ impl Cast { precision: u8, scale: i8, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { Self::cast_floating_point_to_decimal128::(array, precision, scale, eval_mode) } @@ -790,7 +797,7 @@ impl Cast { precision: u8, scale: i8, eval_mode: EvalMode, - ) -> CometResult { + ) -> SparkResult { Self::cast_floating_point_to_decimal128::(array, precision, scale, eval_mode) } @@ -799,7 +806,7 @@ impl Cast { precision: u8, scale: i8, eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where ::Native: AsPrimitive, { @@ -819,7 +826,7 @@ impl Cast { Some(v) => { if Decimal128Type::validate_decimal_precision(v, precision).is_err() { if eval_mode == EvalMode::Ansi { - return Err(CometError::NumericValueOutOfRange { + return Err(SparkError::NumericValueOutOfRange { value: input_value.to_string(), precision, scale, @@ -832,7 +839,7 @@ impl Cast { } None => { if eval_mode == EvalMode::Ansi { - return Err(CometError::NumericValueOutOfRange { + return Err(SparkError::NumericValueOutOfRange { value: input_value.to_string(), precision, scale, @@ -856,7 +863,7 @@ impl Cast { fn spark_cast_float64_to_utf8( from: &dyn Array, _eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { @@ -866,7 +873,7 @@ impl Cast { fn spark_cast_float32_to_utf8( from: &dyn Array, _eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { @@ -878,7 +885,7 @@ impl Cast { eval_mode: EvalMode, from_type: &DataType, to_type: &DataType, - ) -> CometResult { + ) -> SparkResult { match (from_type, to_type) { (DataType::Int64, DataType::Int32) => cast_int_to_int_macro!( array, eval_mode, Int64Type, Int32Type, from_type, i32, "BIGINT", "INT" @@ -908,7 +915,7 @@ impl Cast { fn spark_cast_utf8_to_boolean( from: &dyn Array, eval_mode: EvalMode, - ) -> CometResult + ) -> SparkResult where OffsetSize: OffsetSizeTrait, { @@ -923,7 +930,7 @@ impl Cast { Some(value) => match value.to_ascii_lowercase().trim() { "t" | "true" | "y" | "yes" | "1" => Ok(Some(true)), "f" | "false" | "n" | "no" | "0" => Ok(Some(false)), - _ if eval_mode == EvalMode::Ansi => Err(CometError::CastInvalidValue { + _ if eval_mode == EvalMode::Ansi => Err(SparkError::CastInvalidValue { value: value.to_string(), from_type: "STRING".to_string(), to_type: "BOOLEAN".to_string(), @@ -942,7 +949,7 @@ impl Cast { eval_mode: EvalMode, from_type: &DataType, to_type: &DataType, - ) -> CometResult { + ) -> SparkResult { match (from_type, to_type) { (DataType::Float32, DataType::Int8) => cast_float_to_int16_down!( array, @@ -1079,7 +1086,7 @@ impl Cast { } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toByte -fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> SparkResult> { Ok(cast_string_to_int_with_range_check( str, eval_mode, @@ -1091,7 +1098,7 @@ fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> CometResult> } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toShort -fn cast_string_to_i16(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i16(str: &str, eval_mode: EvalMode) -> SparkResult> { Ok(cast_string_to_int_with_range_check( str, eval_mode, @@ -1103,12 +1110,12 @@ fn cast_string_to_i16(str: &str, eval_mode: EvalMode) -> CometResult } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toInt(IntWrapper intWrapper) -fn cast_string_to_i32(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i32(str: &str, eval_mode: EvalMode) -> SparkResult> { do_cast_string_to_int::(str, eval_mode, "INT", i32::MIN) } /// Equivalent to org.apache.spark.unsafe.types.UTF8String.toLong(LongWrapper intWrapper) -fn cast_string_to_i64(str: &str, eval_mode: EvalMode) -> CometResult> { +fn cast_string_to_i64(str: &str, eval_mode: EvalMode) -> SparkResult> { do_cast_string_to_int::(str, eval_mode, "BIGINT", i64::MIN) } @@ -1118,7 +1125,7 @@ fn cast_string_to_int_with_range_check( type_name: &str, min: i32, max: i32, -) -> CometResult> { +) -> SparkResult> { match do_cast_string_to_int(str, eval_mode, type_name, i32::MIN)? { None => Ok(None), Some(v) if v >= min && v <= max => Ok(Some(v)), @@ -1137,7 +1144,7 @@ fn do_cast_string_to_int< eval_mode: EvalMode, type_name: &str, min_value: T, -) -> CometResult> { +) -> SparkResult> { let trimmed_str = str.trim(); if trimmed_str.is_empty() { return none_or_err(eval_mode, type_name, str); @@ -1221,9 +1228,9 @@ fn do_cast_string_to_int< Ok(Some(result)) } -/// Either return Ok(None) or Err(CometError::CastInvalidValue) depending on the evaluation mode +/// Either return Ok(None) or Err(SparkError::CastInvalidValue) depending on the evaluation mode #[inline] -fn none_or_err(eval_mode: EvalMode, type_name: &str, str: &str) -> CometResult> { +fn none_or_err(eval_mode: EvalMode, type_name: &str, str: &str) -> SparkResult> { match eval_mode { EvalMode::Ansi => Err(invalid_value(str, "STRING", type_name)), _ => Ok(None), @@ -1231,8 +1238,8 @@ fn none_or_err(eval_mode: EvalMode, type_name: &str, str: &str) -> CometResul } #[inline] -fn invalid_value(value: &str, from_type: &str, to_type: &str) -> CometError { - CometError::CastInvalidValue { +fn invalid_value(value: &str, from_type: &str, to_type: &str) -> SparkError { + SparkError::CastInvalidValue { value: value.to_string(), from_type: from_type.to_string(), to_type: to_type.to_string(), @@ -1240,8 +1247,8 @@ fn invalid_value(value: &str, from_type: &str, to_type: &str) -> CometError { } #[inline] -fn cast_overflow(value: &str, from_type: &str, to_type: &str) -> CometError { - CometError::CastOverFlow { +fn cast_overflow(value: &str, from_type: &str, to_type: &str) -> SparkError { + SparkError::CastOverFlow { value: value.to_string(), from_type: from_type.to_string(), to_type: to_type.to_string(), @@ -1329,7 +1336,7 @@ impl PhysicalExpr for Cast { } } -fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult> { +fn timestamp_parser(value: &str, eval_mode: EvalMode) -> SparkResult> { let value = value.trim(); if value.is_empty() { return Ok(None); @@ -1338,7 +1345,7 @@ fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult let patterns = &[ ( Regex::new(r"^\d{4}$").unwrap(), - parse_str_to_year_timestamp as fn(&str) -> CometResult>, + parse_str_to_year_timestamp as fn(&str) -> SparkResult>, ), ( Regex::new(r"^\d{4}-\d{2}$").unwrap(), @@ -1382,7 +1389,7 @@ fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult if timestamp.is_none() { return if eval_mode == EvalMode::Ansi { - Err(CometError::CastInvalidValue { + Err(SparkError::CastInvalidValue { value: value.to_string(), from_type: "STRING".to_string(), to_type: "TIMESTAMP".to_string(), @@ -1394,20 +1401,20 @@ fn timestamp_parser(value: &str, eval_mode: EvalMode) -> CometResult match timestamp { Some(ts) => Ok(Some(ts)), - None => Err(CometError::Internal( + None => Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )), } } -fn parse_ymd_timestamp(year: i32, month: u32, day: u32) -> CometResult> { +fn parse_ymd_timestamp(year: i32, month: u32, day: u32) -> SparkResult> { let datetime = chrono::Utc.with_ymd_and_hms(year, month, day, 0, 0, 0); // Check if datetime is not None let utc_datetime = match datetime.single() { Some(dt) => dt.with_timezone(&chrono::Utc), None => { - return Err(CometError::Internal( + return Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )); } @@ -1424,7 +1431,7 @@ fn parse_hms_timestamp( minute: u32, second: u32, microsecond: u32, -) -> CometResult> { +) -> SparkResult> { let datetime = chrono::Utc.with_ymd_and_hms(year, month, day, hour, minute, second); // Check if datetime is not None @@ -1433,7 +1440,7 @@ fn parse_hms_timestamp( .with_timezone(&chrono::Utc) .with_nanosecond(microsecond * 1000), None => { - return Err(CometError::Internal( + return Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )); } @@ -1442,7 +1449,7 @@ fn parse_hms_timestamp( let result = match utc_datetime { Some(dt) => dt.timestamp_micros(), None => { - return Err(CometError::Internal( + return Err(SparkError::Internal( "Failed to parse timestamp".to_string(), )); } @@ -1451,7 +1458,7 @@ fn parse_hms_timestamp( Ok(Some(result)) } -fn get_timestamp_values(value: &str, timestamp_type: &str) -> CometResult> { +fn get_timestamp_values(value: &str, timestamp_type: &str) -> SparkResult> { let values: Vec<_> = value .split(|c| c == 'T' || c == '-' || c == ':' || c == '.') .collect(); @@ -1471,7 +1478,7 @@ fn get_timestamp_values(value: &str, timestamp_type: &str) -> CometResult