From 725260576669b65ae1ba945b5b7ecdf875d7ecfa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 22 Aug 2025 10:10:23 -0600 Subject: [PATCH 01/37] Access Spark configs from native code --- native/core/src/execution/jni_api.rs | 34 ++++++++++--------- native/core/src/execution/mod.rs | 1 + native/core/src/execution/spark_config.rs | 34 +++++++++++++++++++ .../org/apache/comet/CometExecIterator.scala | 7 ++-- .../main/scala/org/apache/comet/Native.scala | 5 +-- 5 files changed, 56 insertions(+), 25 deletions(-) create mode 100644 native/core/src/execution/spark_config.rs diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index ef5435cbc9..a060e3cea5 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -75,6 +75,9 @@ use crate::execution::spark_plan::SparkPlan; use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace}; +use crate::execution::spark_config::{ + SparkConfig, COMET_DEBUG_ENABLED, COMET_EXPLAIN_NATIVE_ENABLED, COMET_TRACING_ENABLED, +}; use datafusion_comet_proto::spark_operator::operator::OpStruct; use log::info; use once_cell::sync::Lazy; @@ -164,12 +167,20 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( memory_limit: jlong, memory_limit_per_task: jlong, task_attempt_id: jlong, - debug_native: jboolean, - explain_native: jboolean, - tracing_enabled: jboolean, ) -> jlong { try_unwrap_or_throw(&e, |mut env| { - with_trace("createPlan", tracing_enabled != JNI_FALSE, || { + // Deserialize Spark configs + let array = unsafe { JPrimitiveArray::from_raw(serialized_spark_configs) }; + let bytes = env.convert_byte_array(array)?; + let spark_configs = serde::deserialize_config(bytes.as_slice())?; + + // Convert Spark configs to HashMap + let spark_config: HashMap = spark_configs.entries.into_iter().collect(); + let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED); + let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED); + let tracing_enabled = spark_config.get_bool(COMET_TRACING_ENABLED); + + with_trace("createPlan", tracing_enabled, || { // Init JVM classes JVMClasses::init(&mut env); @@ -180,15 +191,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let bytes = env.convert_byte_array(array)?; let spark_plan = serde::deserialize_op(bytes.as_slice())?; - // Deserialize Spark configs - let array = unsafe { JPrimitiveArray::from_raw(serialized_spark_configs) }; - let bytes = env.convert_byte_array(array)?; - let spark_configs = serde::deserialize_config(bytes.as_slice())?; - - // Convert Spark configs to HashMap - let _spark_config_map: HashMap = - spark_configs.entries.into_iter().collect(); - let metrics = Arc::new(jni_new_global_ref!(env, metrics_node)?); // Get the global references of input sources @@ -253,10 +255,10 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( metrics_last_update_time: Instant::now(), plan_creation_time, session_ctx: Arc::new(session), - debug_native: debug_native == 1, - explain_native: explain_native == 1, + debug_native, + explain_native, memory_pool_config, - tracing_enabled: tracing_enabled != JNI_FALSE, + tracing_enabled, }); Ok(Box::into_raw(exec_context) as i64) diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index c55b96f2a9..b8a3d546b3 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -27,6 +27,7 @@ pub(crate) mod sort; pub(crate) mod spark_plan; pub use datafusion_comet_spark_expr::timezone; mod memory_pools; +pub(crate) mod spark_config; pub(crate) mod tracing; pub(crate) mod utils; diff --git a/native/core/src/execution/spark_config.rs b/native/core/src/execution/spark_config.rs new file mode 100644 index 0000000000..7465a1ea9f --- /dev/null +++ b/native/core/src/execution/spark_config.rs @@ -0,0 +1,34 @@ +// 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. + +use std::collections::HashMap; + +pub(crate) const COMET_TRACING_ENABLED: &str = "spark.comet.tracing.enabled"; +pub(crate) const COMET_DEBUG_ENABLED: &str = "spark.comet.debug.enabled"; +pub(crate) const COMET_EXPLAIN_NATIVE_ENABLED: &str = "spark.comet.explain.native.enabled"; + +pub(crate) trait SparkConfig { + fn get_bool(&self, name: &str) -> bool; +} + +impl SparkConfig for HashMap { + fn get_bool(&self, name: &str) -> bool { + self.get(name) + .and_then(|str_val| str_val.parse::().ok()) + .unwrap_or(false) + } +} diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 67d044f8c5..700e786e35 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -30,7 +30,7 @@ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.comet.CometMetricNode import org.apache.spark.sql.vectorized._ -import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_POOL_TYPE, COMET_EXPLAIN_NATIVE_ENABLED, COMET_METRICS_UPDATE_INTERVAL} +import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_EXEC_MEMORY_POOL_TYPE, COMET_METRICS_UPDATE_INTERVAL} import org.apache.comet.Tracing.withTrace import org.apache.comet.serde.Config.ConfigMap import org.apache.comet.vector.NativeUtil @@ -108,10 +108,7 @@ class CometExecIterator( memoryPoolType = COMET_EXEC_MEMORY_POOL_TYPE.get(), memoryLimit, memoryLimitPerTask = getMemoryLimitPerTask(conf), - taskAttemptId = TaskContext.get().taskAttemptId, - debug = COMET_DEBUG_ENABLED.get(), - explain = COMET_EXPLAIN_NATIVE_ENABLED.get(), - tracingEnabled) + taskAttemptId = TaskContext.get().taskAttemptId) } private var nextBatch: Option[ColumnarBatch] = None diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 7430a4322c..a269993bb1 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -65,10 +65,7 @@ class Native extends NativeBase { memoryPoolType: String, memoryLimit: Long, memoryLimitPerTask: Long, - taskAttemptId: Long, - debug: Boolean, - explain: Boolean, - tracingEnabled: Boolean): Long + taskAttemptId: Long): Long // scalastyle:on /** From d084cfa0e192d7655587ce1d3bb7e6038fc2bd7f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 22 Aug 2025 10:16:35 -0600 Subject: [PATCH 02/37] code cleanup --- dev/diffs/4.0.0.diff | 153 +++++++++++++++++++++++---- native/core/src/execution/jni_api.rs | 4 +- 2 files changed, 134 insertions(+), 23 deletions(-) diff --git a/dev/diffs/4.0.0.diff b/dev/diffs/4.0.0.diff index 7d2dc790d3..6799999b8a 100644 --- a/dev/diffs/4.0.0.diff +++ b/dev/diffs/4.0.0.diff @@ -700,10 +700,10 @@ index 9c529d14221..069b7c5adeb 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5eb3fa17ca8 +index 00000000000..5691536c114 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,43 @@ +@@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -732,6 +732,8 @@ index 00000000000..5eb3fa17ca8 + * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). + */ +case class IgnoreComet(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") +case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") + +/** @@ -1055,7 +1057,7 @@ index ad424b3a7cc..4ece0117a34 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index b3fce19979e..345acb4811a 100644 +index b3fce19979e..67edf5eb91c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1524,7 +1524,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1084,11 +1086,24 @@ index b3fce19979e..345acb4811a 100644 test("SPARK-39175: Query context of Cast should be serialized to executors" + - " when WSCG is off") { + " when WSCG is off", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/2218")) { withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", SQLConf.ANSI_ENABLED.key -> "true") { withTable("t") { -@@ -4497,7 +4500,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark +@@ -4490,14 +4493,20 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + assert(ex.isInstanceOf[SparkNumberFormatException] || + ex.isInstanceOf[SparkDateTimeException] || + ex.isInstanceOf[SparkRuntimeException]) +- assert(ex.getMessage.contains(query)) ++ ++ if (!isCometEnabled) { ++ // Comet's error message does not include the original SQL query ++ // https://github.com/apache/datafusion-comet/issues/2215 ++ assert(ex.getMessage.contains(query)) ++ } + } + } + } } test("SPARK-39190,SPARK-39208,SPARK-39210: Query context of decimal overflow error should " + @@ -1277,11 +1292,26 @@ index 2e33f6505ab..e1e93ab3bad 100644 } withTable("t1", "t2") { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala +index fee375db10a..8c2c24e2c5f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala +@@ -33,7 +33,9 @@ import org.apache.spark.sql.types._ + import org.apache.spark.types.variant._ + import org.apache.spark.unsafe.types.{UTF8String, VariantVal} + +-class VariantShreddingSuite extends QueryTest with SharedSparkSession with ParquetTest { ++class VariantShreddingSuite extends QueryTest with SharedSparkSession with ParquetTest ++ // TODO enable tests once https://github.com/apache/datafusion-comet/issues/2209 is fixed ++ with IgnoreCometSuite { + def parseJson(s: String): VariantVal = { + val v = VariantBuilder.parseJson(s, false) + new VariantVal(v.getValue, v.getMetadata) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala -index 11e9547dfc5..df5678c8d82 100644 +index 11e9547dfc5..be9ae40ab3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala -@@ -20,7 +20,7 @@ package org.apache.spark.sql.collation +@@ -20,10 +20,11 @@ package org.apache.spark.sql.collation import scala.jdk.CollectionConverters.MapHasAsJava import org.apache.spark.SparkException @@ -1290,7 +1320,21 @@ index 11e9547dfc5..df5678c8d82 100644 import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.CollationFactory -@@ -1505,7 +1505,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ++import org.apache.spark.sql.comet.{CometBroadcastHashJoinExec, CometHashJoinExec, CometSortMergeJoinExec} + import org.apache.spark.sql.connector.{DatasourceV2SQLBase, FakeV2ProviderWithCustomSchema} + import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable} + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper +@@ -55,7 +56,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { + assert( + collectFirst(queryPlan) { + case _: SortMergeJoinExec => assert(isSortMergeForced) ++ case _: CometSortMergeJoinExec => assert(isSortMergeForced) + case _: HashJoin => assert(!isSortMergeForced) ++ case _: CometHashJoinExec | _: CometBroadcastHashJoinExec => assert(!isSortMergeForced) + }.nonEmpty + ) + } +@@ -1505,7 +1508,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } @@ -1301,7 +1345,23 @@ index 11e9547dfc5..df5678c8d82 100644 val t1 = "T_1" val t2 = "T_2" -@@ -1815,7 +1817,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { +@@ -1611,6 +1616,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { + } else { + assert(!collectFirst(queryPlan) { + case b: BroadcastHashJoinExec => b.leftKeys.head ++ case b: CometBroadcastHashJoinExec => b.leftKeys.head + }.head.isInstanceOf[ArrayTransform]) + } + } +@@ -1676,6 +1682,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { + } else { + assert(!collectFirst(queryPlan) { + case b: BroadcastHashJoinExec => b.leftKeys.head ++ case b: CometBroadcastHashJoinExec => b.leftKeys.head + }.head.isInstanceOf[ArrayTransform]) + } + } +@@ -1815,7 +1822,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } @@ -2636,10 +2696,23 @@ index 22839d3f0d2..7e66d100e90 100644 checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index bba71f1c48d..4f33ce4b3f2 100644 +index bba71f1c48d..38c60ee2584 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -@@ -1060,7 +1060,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -996,7 +996,11 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + Seq(Some("A"), Some("A"), None).toDF().repartition(1) + .write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) +- checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), df) ++ // Similar to Spark's vectorized reader, Comet doesn't do row-level filtering but relies ++ // on Spark to apply the data filters after columnar batches are returned ++ if (!isCometEnabled) { ++ checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), df) ++ } + } + } + } +@@ -1060,7 +1064,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS checkAnswer(readParquet(schema2, path), df) } @@ -2649,7 +2722,7 @@ index bba71f1c48d..4f33ce4b3f2 100644 val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" checkAnswer(readParquet(schema1, path), df) val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" -@@ -1084,7 +1085,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1084,7 +1089,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") df.write.parquet(path.toString) @@ -3330,27 +3403,34 @@ index 86c4e49f6f6..2e639e5f38d 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 f0f3f94b811..486a436afb2 100644 +index f0f3f94b811..d64e4e54e22 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 -@@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, Suite, Tag} +@@ -27,13 +27,14 @@ import scala.jdk.CollectionConverters._ + import scala.language.implicitConversions + import scala.util.control.NonFatal + ++import org.apache.comet.CometConf + import org.apache.hadoop.fs.Path + import org.scalactic.source.Position + import org.scalatest.{BeforeAndAfterAll, Suite, Tag} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, IgnoreCometNativeIcebergCompat, IgnoreCometNativeScan, Row} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE -@@ -42,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase +@@ -42,6 +43,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.classic.{ClassicConversions, ColumnConversions, ColumnNodeToExpressionConverter, DataFrame, Dataset, SparkSession, SQLImplicits} -+import org.apache.spark.sql.comet._ ++import org.apache.spark.sql.comet.{CometFilterExec, CometProjectExec} import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -128,7 +129,11 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with +@@ -128,7 +130,28 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with } } } else { @@ -3358,12 +3438,29 @@ index f0f3f94b811..486a436afb2 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + } else { -+ super.test(testName, testTags: _*)(testFun) ++ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) ++ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || ++ cometScanImpl == CometConf.SCAN_AUTO ++ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || ++ cometScanImpl == CometConf.SCAN_AUTO ++ if (isCometEnabled && isNativeIcebergCompat && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { ++ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) ++ } else if (isCometEnabled && isNativeDataFusion && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) ++ } else if (isCometEnabled && (isNativeDataFusion || isNativeIcebergCompat) && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", ++ testTags: _*)(testFun) ++ } else { ++ super.test(testName, testTags: _*)(testFun) ++ } + } } } -@@ -248,8 +253,33 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +271,33 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3397,7 +3494,7 @@ index f0f3f94b811..486a436afb2 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +481,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +499,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3509,6 +3606,20 @@ index 4b27082e188..09f591dfed3 100644 Utils.withContextClassLoader(Utils.getSparkClassLoader) { withUserDefinedFunction(udfInfo.funcName -> false) { val sparkClassLoader = Thread.currentThread().getContextClassLoader +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +index cc7bb193731..06555d48da7 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +@@ -818,7 +818,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter + } + } + +- test("SPARK-30201 HiveOutputWriter standardOI should use ObjectInspectorCopyOption.DEFAULT") { ++ test("SPARK-30201 HiveOutputWriter standardOI should use ObjectInspectorCopyOption.DEFAULT", ++ IgnoreComet("Comet does not support reading non UTF-8 strings")) { + withTable("t1", "t2") { + withTempDir { dir => + val file = new File(dir, "test.hex") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala index b67370f6eb9..746b3974b29 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index a060e3cea5..1694d81ea7 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -173,9 +173,9 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let array = unsafe { JPrimitiveArray::from_raw(serialized_spark_configs) }; let bytes = env.convert_byte_array(array)?; let spark_configs = serde::deserialize_config(bytes.as_slice())?; - - // Convert Spark configs to HashMap let spark_config: HashMap = spark_configs.entries.into_iter().collect(); + + // Access Spark configs let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED); let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED); let tracing_enabled = spark_config.get_bool(COMET_TRACING_ENABLED); From 48379352f6f684e74e7052659ae31f0dcfe0cc3b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 22 Aug 2025 10:18:18 -0600 Subject: [PATCH 03/37] revert --- dev/diffs/4.0.0.diff | 153 ++++++------------------------------------- 1 file changed, 21 insertions(+), 132 deletions(-) diff --git a/dev/diffs/4.0.0.diff b/dev/diffs/4.0.0.diff index 6799999b8a..7d2dc790d3 100644 --- a/dev/diffs/4.0.0.diff +++ b/dev/diffs/4.0.0.diff @@ -700,10 +700,10 @@ index 9c529d14221..069b7c5adeb 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5691536c114 +index 00000000000..5eb3fa17ca8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -732,8 +732,6 @@ index 00000000000..5691536c114 + * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). + */ +case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") +case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") + +/** @@ -1057,7 +1055,7 @@ index ad424b3a7cc..4ece0117a34 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index b3fce19979e..67edf5eb91c 100644 +index b3fce19979e..345acb4811a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1524,7 +1524,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1086,24 +1084,11 @@ index b3fce19979e..67edf5eb91c 100644 test("SPARK-39175: Query context of Cast should be serialized to executors" + - " when WSCG is off") { + " when WSCG is off", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/2218")) { ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/551")) { withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", SQLConf.ANSI_ENABLED.key -> "true") { withTable("t") { -@@ -4490,14 +4493,20 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark - assert(ex.isInstanceOf[SparkNumberFormatException] || - ex.isInstanceOf[SparkDateTimeException] || - ex.isInstanceOf[SparkRuntimeException]) -- assert(ex.getMessage.contains(query)) -+ -+ if (!isCometEnabled) { -+ // Comet's error message does not include the original SQL query -+ // https://github.com/apache/datafusion-comet/issues/2215 -+ assert(ex.getMessage.contains(query)) -+ } - } - } - } +@@ -4497,7 +4500,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } test("SPARK-39190,SPARK-39208,SPARK-39210: Query context of decimal overflow error should " + @@ -1292,26 +1277,11 @@ index 2e33f6505ab..e1e93ab3bad 100644 } withTable("t1", "t2") { -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala -index fee375db10a..8c2c24e2c5f 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantShreddingSuite.scala -@@ -33,7 +33,9 @@ import org.apache.spark.sql.types._ - import org.apache.spark.types.variant._ - import org.apache.spark.unsafe.types.{UTF8String, VariantVal} - --class VariantShreddingSuite extends QueryTest with SharedSparkSession with ParquetTest { -+class VariantShreddingSuite extends QueryTest with SharedSparkSession with ParquetTest -+ // TODO enable tests once https://github.com/apache/datafusion-comet/issues/2209 is fixed -+ with IgnoreCometSuite { - def parseJson(s: String): VariantVal = { - val v = VariantBuilder.parseJson(s, false) - new VariantVal(v.getValue, v.getMetadata) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala -index 11e9547dfc5..be9ae40ab3d 100644 +index 11e9547dfc5..df5678c8d82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala -@@ -20,10 +20,11 @@ package org.apache.spark.sql.collation +@@ -20,7 +20,7 @@ package org.apache.spark.sql.collation import scala.jdk.CollectionConverters.MapHasAsJava import org.apache.spark.SparkException @@ -1320,21 +1290,7 @@ index 11e9547dfc5..be9ae40ab3d 100644 import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.CollationFactory -+import org.apache.spark.sql.comet.{CometBroadcastHashJoinExec, CometHashJoinExec, CometSortMergeJoinExec} - import org.apache.spark.sql.connector.{DatasourceV2SQLBase, FakeV2ProviderWithCustomSchema} - import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable} - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper -@@ -55,7 +56,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { - assert( - collectFirst(queryPlan) { - case _: SortMergeJoinExec => assert(isSortMergeForced) -+ case _: CometSortMergeJoinExec => assert(isSortMergeForced) - case _: HashJoin => assert(!isSortMergeForced) -+ case _: CometHashJoinExec | _: CometBroadcastHashJoinExec => assert(!isSortMergeForced) - }.nonEmpty - ) - } -@@ -1505,7 +1508,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { +@@ -1505,7 +1505,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } @@ -1345,23 +1301,7 @@ index 11e9547dfc5..be9ae40ab3d 100644 val t1 = "T_1" val t2 = "T_2" -@@ -1611,6 +1616,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { - } else { - assert(!collectFirst(queryPlan) { - case b: BroadcastHashJoinExec => b.leftKeys.head -+ case b: CometBroadcastHashJoinExec => b.leftKeys.head - }.head.isInstanceOf[ArrayTransform]) - } - } -@@ -1676,6 +1682,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { - } else { - assert(!collectFirst(queryPlan) { - case b: BroadcastHashJoinExec => b.leftKeys.head -+ case b: CometBroadcastHashJoinExec => b.leftKeys.head - }.head.isInstanceOf[ArrayTransform]) - } - } -@@ -1815,7 +1822,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { +@@ -1815,7 +1817,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } @@ -2696,23 +2636,10 @@ index 22839d3f0d2..7e66d100e90 100644 checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index bba71f1c48d..38c60ee2584 100644 +index bba71f1c48d..4f33ce4b3f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -@@ -996,7 +996,11 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - Seq(Some("A"), Some("A"), None).toDF().repartition(1) - .write.parquet(path.getAbsolutePath) - val df = spark.read.parquet(path.getAbsolutePath) -- checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), df) -+ // Similar to Spark's vectorized reader, Comet doesn't do row-level filtering but relies -+ // on Spark to apply the data filters after columnar batches are returned -+ if (!isCometEnabled) { -+ checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), df) -+ } - } - } - } -@@ -1060,7 +1064,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1060,7 +1060,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS checkAnswer(readParquet(schema2, path), df) } @@ -2722,7 +2649,7 @@ index bba71f1c48d..38c60ee2584 100644 val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" checkAnswer(readParquet(schema1, path), df) val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" -@@ -1084,7 +1089,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1084,7 +1085,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") df.write.parquet(path.toString) @@ -3403,34 +3330,27 @@ index 86c4e49f6f6..2e639e5f38d 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 f0f3f94b811..d64e4e54e22 100644 +index f0f3f94b811..486a436afb2 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 -@@ -27,13 +27,14 @@ import scala.jdk.CollectionConverters._ - import scala.language.implicitConversions - import scala.util.control.NonFatal - -+import org.apache.comet.CometConf - import org.apache.hadoop.fs.Path - import org.scalactic.source.Position - import org.scalatest.{BeforeAndAfterAll, Suite, Tag} +@@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, Suite, Tag} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, IgnoreCometNativeIcebergCompat, IgnoreCometNativeScan, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, Row} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE -@@ -42,6 +43,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase +@@ -42,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.classic.{ClassicConversions, ColumnConversions, ColumnNodeToExpressionConverter, DataFrame, Dataset, SparkSession, SQLImplicits} -+import org.apache.spark.sql.comet.{CometFilterExec, CometProjectExec} ++import org.apache.spark.sql.comet._ import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -128,7 +130,28 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with +@@ -128,7 +129,11 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with } } } else { @@ -3438,29 +3358,12 @@ index f0f3f94b811..d64e4e54e22 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + } else { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isCometEnabled && isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ } else if (isCometEnabled && isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ } else if (isCometEnabled && (isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ } else { -+ super.test(testName, testTags: _*)(testFun) -+ } ++ super.test(testName, testTags: _*)(testFun) + } } } -@@ -248,8 +271,33 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +253,33 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3494,7 +3397,7 @@ index f0f3f94b811..d64e4e54e22 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +499,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +481,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3606,20 +3509,6 @@ index 4b27082e188..09f591dfed3 100644 Utils.withContextClassLoader(Utils.getSparkClassLoader) { withUserDefinedFunction(udfInfo.funcName -> false) { val sparkClassLoader = Thread.currentThread().getContextClassLoader -diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala -index cc7bb193731..06555d48da7 100644 ---- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala -+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala -@@ -818,7 +818,8 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter - } - } - -- test("SPARK-30201 HiveOutputWriter standardOI should use ObjectInspectorCopyOption.DEFAULT") { -+ test("SPARK-30201 HiveOutputWriter standardOI should use ObjectInspectorCopyOption.DEFAULT", -+ IgnoreComet("Comet does not support reading non UTF-8 strings")) { - withTable("t1", "t2") { - withTempDir { dir => - val file = new File(dir, "test.hex") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala index b67370f6eb9..746b3974b29 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala From ad9c9b8df007b54889e702cc27bfe306a4f85b05 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 09:18:00 -0600 Subject: [PATCH 04/37] debug --- dev/benchmarks/comet-tpch.sh | 12 +- native/Cargo.lock | 524 ++++++++++++------ native/Cargo.toml | 4 +- native/core/Cargo.toml | 1 + native/core/src/errors.rs | 8 +- native/core/src/execution/jni_api.rs | 4 + .../execution/memory_pools/logging_pool.rs | 77 +++ native/core/src/execution/memory_pools/mod.rs | 1 + 8 files changed, 454 insertions(+), 177 deletions(-) create mode 100644 native/core/src/execution/memory_pools/logging_pool.rs diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index e1f6f969ff..b895333185 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -22,25 +22,25 @@ $SPARK_HOME/sbin/stop-master.sh $SPARK_HOME/sbin/stop-worker.sh $SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER +RUST_BACKTRACE=1 $SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER -$SPARK_HOME/bin/spark-submit \ +RUST_BACKTRACE=1 $SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ --jars $COMET_JAR \ --driver-class-path $COMET_JAR \ --conf spark.driver.memory=8G \ --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ + --conf spark.executor.cores=1 \ + --conf spark.cores.max=1 \ --conf spark.executor.memory=16g \ --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ + --conf spark.memory.offHeap.size=1g \ --conf spark.eventLog.enabled=true \ --conf spark.driver.extraClassPath=$COMET_JAR \ --conf spark.executor.extraClassPath=$COMET_JAR \ --conf spark.plugins=org.apache.spark.CometPlugin \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.exec.replaceSortMergeJoin=true \ + --conf spark.comet.exec.replaceSortMergeJoin=false \ --conf spark.comet.expression.allowIncompatible=true \ --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ diff --git a/native/Cargo.lock b/native/Cargo.lock index c707a72537..9ab1faf54c 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1390,8 +1390,7 @@ dependencies = [ [[package]] name = "datafusion" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "481d0c1cad7606cee11233abcdff8eec46e43dd25abda007db6d5d26ae8483c4" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "arrow-ipc", @@ -1401,24 +1400,24 @@ dependencies = [ "chrono", "datafusion-catalog", "datafusion-catalog-listing", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", "datafusion-datasource", "datafusion-datasource-csv", "datafusion-datasource-json", "datafusion-datasource-parquet", - "datafusion-execution", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-functions", - "datafusion-functions-aggregate", - "datafusion-functions-nested", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-nested 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-functions-table", "datafusion-functions-window", "datafusion-optimizer", - "datafusion-physical-expr", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-expr-adapter", - "datafusion-physical-expr-common", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-session", @@ -1441,18 +1440,17 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d70327e81ab3a1f5832d8b372d55fa607851d7cea6d1f8e65ff0c98fcc32d222" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "datafusion-session", "datafusion-sql", @@ -1467,18 +1465,17 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "268819e6bb20ba70a664abddc20deac604f30d3267f8c91847064542a8c0720c" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1503,7 +1500,7 @@ dependencies = [ "datafusion-comet-objectstore-hdfs", "datafusion-comet-proto", "datafusion-comet-spark-expr", - "datafusion-functions-nested", + "datafusion-functions-nested 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "datafusion-spark", "futures", "hdfs-sys", @@ -1603,6 +1600,27 @@ name = "datafusion-common" version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "054873d5563f115f83ef4270b560ac2ce4de713905e825a40cac49d6ff348254" +dependencies = [ + "ahash", + "arrow", + "arrow-ipc", + "base64", + "chrono", + "half", + "hashbrown 0.14.5", + "indexmap", + "libc", + "log", + "paste", + "sqlparser", + "tokio", + "web-time", +] + +[[package]] +name = "datafusion-common" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "ahash", "arrow", @@ -1625,8 +1643,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8a1d1bc69aaaadb8008b65329ed890b33e845dc063225c190f77b20328fbe1d" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "futures", "log", @@ -1636,20 +1653,19 @@ dependencies = [ [[package]] name = "datafusion-datasource" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d855160469020982880fd9bd0962e033d2f4728f56f85a83d8c90785638b6519" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "bytes", "chrono", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-expr-adapter", - "datafusion-physical-expr-common", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1667,20 +1683,19 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ec3aa7575378d23aae96b955b5233bea6f9d461648174f6ccc8f3c160f2b7a7" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1692,20 +1707,19 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00cfb8f33e2864eeb3188b6818acf5546d56a5a487d423cce9b684a554caabfa" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1717,22 +1731,21 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab3bfb48fb4ff42ac1485a12ea56434eaab53f7da8f00b2443b1a3d35a0b6d10" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate", - "datafusion-physical-expr", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-expr-adapter", - "datafusion-physical-expr-common", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-pruning", @@ -1753,6 +1766,11 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2fbf41013cf55c2369b5229594898e8108c8a1beeb49d97feb5e0cce9933eb8f" +[[package]] +name = "datafusion-doc" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" + [[package]] name = "datafusion-execution" version = "50.0.0" @@ -1762,8 +1780,27 @@ dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common", - "datafusion-expr", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "futures", + "log", + "object_store", + "parking_lot", + "rand", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-execution" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "arrow", + "async-trait", + "dashmap", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "futures", "log", "object_store", @@ -1782,12 +1819,32 @@ dependencies = [ "arrow", "async-trait", "chrono", - "datafusion-common", - "datafusion-doc", - "datafusion-expr-common", - "datafusion-functions-aggregate-common", - "datafusion-functions-window-common", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions-window-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "indexmap", + "paste", + "serde_json", + "sqlparser", +] + +[[package]] +name = "datafusion-expr" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "arrow", + "async-trait", + "chrono", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-window-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "indexmap", "paste", "serde_json", @@ -1801,7 +1858,19 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4a45bee7d2606bfb41ceb1d904ba7cecf69bd5a6f8f3e6c57c3f5a83d84bdd97" dependencies = [ "arrow", - "datafusion-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "indexmap", + "itertools 0.14.0", + "paste", +] + +[[package]] +name = "datafusion-expr-common" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "arrow", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "indexmap", "itertools 0.14.0", "paste", @@ -1812,6 +1881,30 @@ name = "datafusion-functions" version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c7e1c532ff9d14f291160bca23e55ffd4899800301dd2389786c2f02d76904a" +dependencies = [ + "arrow", + "arrow-buffer", + "base64", + "chrono", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-execution 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-macros 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "hex", + "itertools 0.14.0", + "log", + "rand", + "regex", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-functions" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "arrow-buffer", @@ -1819,12 +1912,12 @@ dependencies = [ "blake2", "blake3", "chrono", - "datafusion-common", - "datafusion-doc", - "datafusion-execution", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-macros", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "hex", "itertools 0.14.0", "log", @@ -1844,14 +1937,34 @@ checksum = "b05d47426645aef1e73b1a034c75ab2401bc504175feb191accbe211ec24a342" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-doc", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate-common", - "datafusion-macros", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-execution 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-macros 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-physical-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "half", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-aggregate" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "half", "log", "paste", @@ -1865,9 +1978,21 @@ checksum = "05c99f648b2b1743de0c1c19eef07e8cc5a085237f172b2e20bf6934e0a804e4" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-expr-common", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", +] + +[[package]] +name = "datafusion-functions-aggregate-common" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", ] [[package]] @@ -1878,15 +2003,36 @@ checksum = "4227782023f4fb68d3d5c5eb190665212f43c9a0b437553e4b938b379aff6cf6" dependencies = [ "arrow", "arrow-ord", - "datafusion-common", - "datafusion-doc", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions", - "datafusion-functions-aggregate", - "datafusion-functions-aggregate-common", - "datafusion-macros", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-execution 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions-aggregate 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-macros 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "itertools 0.14.0", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-nested" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "arrow", + "arrow-ord", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "itertools 0.14.0", "log", "paste", @@ -1895,14 +2041,13 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d902b1769f69058236e89f04f3bff2cf62f24311adb7bf3c6c3e945c9451076" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common", - "datafusion-expr", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "parking_lot", "paste", @@ -1911,17 +2056,16 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b8ee43974c92eb9920fe8e97e0fab48675e93b062abcb48bef4c1d4305b6ee4" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", - "datafusion-common", - "datafusion-doc", - "datafusion-expr", - "datafusion-functions-window-common", - "datafusion-macros", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-window-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "log", "paste", ] @@ -1932,8 +2076,17 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a1e149d36cdd44fb425dc815c5fac55025aa9a592dd65cb3c421881096292c02" dependencies = [ - "datafusion-common", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", +] + +[[package]] +name = "datafusion-functions-window-common" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", ] [[package]] @@ -1942,7 +2095,17 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "07c9faa0cdefb6e6e756482b846397b5c2d84d369e30b009472b9ab9b1430fbd" dependencies = [ - "datafusion-expr", + "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "quote", + "syn 2.0.106", +] + +[[package]] +name = "datafusion-macros" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "quote", "syn 2.0.106", ] @@ -1950,15 +2113,14 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f16a4f7059302ad1de6e97ab0eebb5c34405917b1f80806a30a66e38ad118251" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "chrono", - "datafusion-common", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-physical-expr", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "indexmap", "itertools 0.14.0", "log", @@ -1974,11 +2136,33 @@ checksum = "10bb87a605d8ce9672d5347c0293c12211b0c03923fc12fbdc665fe76e6f9e01" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-functions-aggregate-common", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "half", + "hashbrown 0.14.5", + "indexmap", + "itertools 0.14.0", + "log", + "parking_lot", + "paste", + "petgraph 0.8.2", +] + +[[package]] +name = "datafusion-physical-expr" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "half", "hashbrown 0.14.5", "indexmap", @@ -1992,15 +2176,14 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-adapter" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2da3a7429a555dd5ff0bec4d24bd5532ec43876764088da635cad55b2f178dc2" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", - "datafusion-common", - "datafusion-expr", - "datafusion-functions", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "itertools 0.14.0", ] @@ -2012,8 +2195,21 @@ checksum = "845eb44ef1e04d2a15c6d955cb146b40a41814a7be4377f0a541857d3e257d6f" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-expr-common", + "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "hashbrown 0.14.5", + "itertools 0.14.0", +] + +[[package]] +name = "datafusion-physical-expr-common" +version = "50.0.0" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "hashbrown 0.14.5", "itertools 0.14.0", ] @@ -2021,16 +2217,15 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32b9b648ee2785722c79eae366528e52e93ece6808aef9297cf8e5521de381da" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "datafusion-pruning", "itertools 0.14.0", @@ -2040,8 +2235,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e6688d17b78104e169d7069749832c20ff50f112be853d2c058afe46c889064" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "ahash", "arrow", @@ -2049,14 +2243,14 @@ dependencies = [ "arrow-schema", "async-trait", "chrono", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate-common", - "datafusion-functions-window-common", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions-window-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "futures", "half", "hashbrown 0.14.5", @@ -2071,16 +2265,15 @@ dependencies = [ [[package]] name = "datafusion-pruning" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a893a46c56f5f190085e13949eb8ec163672c7ec2ac33bdb82c84572e71ca73" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "arrow-schema", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-datasource", - "datafusion-expr-common", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "itertools 0.14.0", "log", @@ -2089,17 +2282,16 @@ dependencies = [ [[package]] name = "datafusion-session" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8b62684c7a1db6121a8c83100209cffa1e664a8d9ced87e1a32f8cdc2fff3c2" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "datafusion-physical-plan", "datafusion-sql", "futures", @@ -2113,18 +2305,17 @@ dependencies = [ [[package]] name = "datafusion-spark" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b217679a4f1bbfae8ec97143950c42940eebba7c566b4141184fa8f81d296db1" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "chrono", "crc32fast", "datafusion-catalog", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions", - "datafusion-macros", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "log", "sha1", "url", @@ -2134,13 +2325,12 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "50.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f09cff94b8242843e1da5d069e9d2cfc53807f1f00b1c0da78c297f47c21456e" +source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "bigdecimal", - "datafusion-common", - "datafusion-expr", + "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", "indexmap", "log", "regex", diff --git a/native/Cargo.toml b/native/Cargo.toml index 65b9a7eaf2..bf89c389bb 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -38,8 +38,8 @@ arrow = { version = "56.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "=56.0.0", default-features = false, features = ["experimental"] } -datafusion = { version = "50.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } -datafusion-spark = { version = "50.0.0" } +datafusion = { git = "https://github.com/andygrove/datafusion", rev = "720b55a24918ac60a04dd377d7d9ab594741e7c5", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion-spark = { git = "https://github.com/andygrove/datafusion", rev = "720b55a24918ac60a04dd377d7d9ab594741e7c5" } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 38101b31ed..a5d11aed65 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -94,6 +94,7 @@ hex = "0.4.3" datafusion-functions-nested = { version = "50.0.0" } [features] +backtrace = ["datafusion/backtrace"] default = [] hdfs = ["datafusion-comet-objectstore-hdfs"] hdfs-opendal = ["opendal", "object_store_opendal", "hdfs-sys"] diff --git a/native/core/src/errors.rs b/native/core/src/errors.rs index b3241477b8..32f05c3c8e 100644 --- a/native/core/src/errors.rs +++ b/native/core/src/errors.rs @@ -42,6 +42,7 @@ use datafusion_comet_spark_expr::SparkError; use jni::objects::{GlobalRef, JThrowable, JValue}; use jni::JNIEnv; use lazy_static::lazy_static; +use log::error; use parquet::errors::ParquetError; use thiserror::Error; @@ -146,9 +147,11 @@ pub enum CometError { pub fn init() { std::panic::set_hook(Box::new(|_panic_info| { + let x = std::backtrace::Backtrace::force_capture().to_string(); + error!("PANIC: {x}"); + // Capture the backtrace for a panic - *PANIC_BACKTRACE.lock().unwrap() = - Some(std::backtrace::Backtrace::force_capture().to_string()); + *PANIC_BACKTRACE.lock().unwrap() = Some(x); })); } @@ -359,6 +362,7 @@ pub fn unwrap_or_throw_default( CometError::Panic { msg: _ } => PANIC_BACKTRACE.lock().unwrap().take(), _ => None, }; + error!("Comet native call failed: {err:?} {backtrace:?}"); throw_exception(env, &err, backtrace); T::default() } diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 52b8eb6a30..99e67df8bb 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -78,6 +78,7 @@ use crate::execution::spark_plan::SparkPlan; use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace}; +use crate::execution::memory_pools::logging_pool::LoggingPool; use datafusion_comet_proto::spark_operator::operator::OpStruct; use log::info; use once_cell::sync::Lazy; @@ -218,6 +219,9 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let memory_pool = create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id); + // TODO only if debug option set + let memory_pool = Arc::new(LoggingPool::new(memory_pool)); + // Get local directories for storing spill files let local_dirs_array = JObjectArray::from_raw(local_dirs); let num_local_dirs = env.get_array_length(&local_dirs_array)?; diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs new file mode 100644 index 0000000000..cb10f39792 --- /dev/null +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -0,0 +1,77 @@ +// 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. + +use datafusion::execution::memory_pool::{MemoryPool, MemoryReservation}; +use log::trace; +use std::sync::Arc; + +#[derive(Debug)] +pub(crate) struct LoggingPool { + pool: Arc, +} + +impl LoggingPool { + pub fn new(pool: Arc) -> Self { + Self { pool } + } +} + +impl MemoryPool for LoggingPool { + fn grow(&self, reservation: &MemoryReservation, additional: usize) { + trace!( + "MemoryPool[{}].grow({})", + reservation.consumer().name(), + reservation.size() + ); + self.pool.grow(reservation, additional); + } + + fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { + trace!( + "MemoryPool[{}].shrink({})", + reservation.consumer().name(), + reservation.size() + ); + self.pool.shrink(reservation, shrink); + } + + fn try_grow( + &self, + reservation: &MemoryReservation, + additional: usize, + ) -> datafusion::common::Result<()> { + let result = self.pool.try_grow(reservation, additional); + if result.is_ok() { + trace!( + "MemoryPool[{}].try_grow({}) returning Ok", + reservation.consumer().name(), + reservation.size() + ); + } else { + trace!( + "MemoryPool[{}].try_grow({}) returning Err", + reservation.consumer().name(), + reservation.size() + ); + } + result + } + + fn reserved(&self) -> usize { + self.pool.reserved() + } +} diff --git a/native/core/src/execution/memory_pools/mod.rs b/native/core/src/execution/memory_pools/mod.rs index 3e40dc6923..f19813fe39 100644 --- a/native/core/src/execution/memory_pools/mod.rs +++ b/native/core/src/execution/memory_pools/mod.rs @@ -17,6 +17,7 @@ mod config; mod fair_pool; +pub mod logging_pool; mod task_shared; mod unified_pool; From f3bb412b11cd25b0c15337676069fbb9656e5eaa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 09:19:33 -0600 Subject: [PATCH 05/37] use df release --- native/Cargo.lock | 558 +++++++++++++++------------------------------- native/Cargo.toml | 4 +- 2 files changed, 186 insertions(+), 376 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 9ab1faf54c..11d3de3f08 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1389,8 +1389,9 @@ dependencies = [ [[package]] name = "datafusion" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4016a135c11820d9c9884a1f7924d5456c563bd3657b7d691a6e7b937a452df7" dependencies = [ "arrow", "arrow-ipc", @@ -1400,24 +1401,24 @@ dependencies = [ "chrono", "datafusion-catalog", "datafusion-catalog-listing", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-datasource-csv", "datafusion-datasource-json", "datafusion-datasource-parquet", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-nested 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-nested", "datafusion-functions-table", "datafusion-functions-window", "datafusion-optimizer", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr", "datafusion-physical-expr-adapter", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-session", @@ -1439,18 +1440,19 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1721d3973afeb8a0c3f235a79101cc61e4a558dd3f02fdc9ae6c61e882e544d9" dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-plan", "datafusion-session", "datafusion-sql", @@ -1464,18 +1466,19 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44841d3efb0c89c6a5ac6fde5ac61d4f2474a2767f170db6d97300a8b4df8904" dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-datasource", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1500,7 +1503,7 @@ dependencies = [ "datafusion-comet-objectstore-hdfs", "datafusion-comet-proto", "datafusion-comet-spark-expr", - "datafusion-functions-nested 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "datafusion-functions-nested", "datafusion-spark", "futures", "hdfs-sys", @@ -1600,27 +1603,6 @@ name = "datafusion-common" version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "054873d5563f115f83ef4270b560ac2ce4de713905e825a40cac49d6ff348254" -dependencies = [ - "ahash", - "arrow", - "arrow-ipc", - "base64", - "chrono", - "half", - "hashbrown 0.14.5", - "indexmap", - "libc", - "log", - "paste", - "sqlparser", - "tokio", - "web-time", -] - -[[package]] -name = "datafusion-common" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "ahash", "arrow", @@ -1642,8 +1624,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f03fe3936f978fe8e76776d14ad8722e33843b01d81d11707ca72d54d2867787" dependencies = [ "futures", "log", @@ -1652,20 +1635,21 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4543216d2f4fc255780a46ae9e062e50c86ac23ecab6718cc1ba3fe4a8d5a8f2" dependencies = [ "arrow", "async-trait", "bytes", "chrono", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-expr-adapter", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1682,20 +1666,21 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ab662d4692ca5929ce32eb609c6c8a741772537d98363b3efb3bc68148cd530" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1706,20 +1691,21 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7dad4492ba9a2fca417cb211f8f05ffeb7f12a1f0f8e5bdcf548c353ff923779" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1730,22 +1716,23 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2925432ce04847cc09b4789a53fc22b0fdf5f2e73289ad7432759d76c6026e9e" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-physical-expr", "datafusion-physical-expr-adapter", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-pruning", @@ -1766,11 +1753,6 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2fbf41013cf55c2369b5229594898e8108c8a1beeb49d97feb5e0cce9933eb8f" -[[package]] -name = "datafusion-doc" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" - [[package]] name = "datafusion-execution" version = "50.0.0" @@ -1780,27 +1762,8 @@ dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "futures", - "log", - "object_store", - "parking_lot", - "rand", - "tempfile", - "url", -] - -[[package]] -name = "datafusion-execution" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "arrow", - "async-trait", - "dashmap", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr", "futures", "log", "object_store", @@ -1819,32 +1782,12 @@ dependencies = [ "arrow", "async-trait", "chrono", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-functions-window-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "indexmap", - "paste", - "serde_json", - "sqlparser", -] - -[[package]] -name = "datafusion-expr" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "arrow", - "async-trait", - "chrono", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-window-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-doc", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", + "datafusion-physical-expr-common", "indexmap", "paste", "serde_json", @@ -1858,19 +1801,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4a45bee7d2606bfb41ceb1d904ba7cecf69bd5a6f8f3e6c57c3f5a83d84bdd97" dependencies = [ "arrow", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "indexmap", - "itertools 0.14.0", - "paste", -] - -[[package]] -name = "datafusion-expr-common" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "indexmap", "itertools 0.14.0", "paste", @@ -1881,30 +1812,6 @@ name = "datafusion-functions" version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c7e1c532ff9d14f291160bca23e55ffd4899800301dd2389786c2f02d76904a" -dependencies = [ - "arrow", - "arrow-buffer", - "base64", - "chrono", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-execution 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-macros 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "hex", - "itertools 0.14.0", - "log", - "rand", - "regex", - "unicode-segmentation", - "uuid", -] - -[[package]] -name = "datafusion-functions" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" dependencies = [ "arrow", "arrow-buffer", @@ -1912,12 +1819,12 @@ dependencies = [ "blake2", "blake3", "chrono", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", "hex", "itertools 0.14.0", "log", @@ -1937,34 +1844,14 @@ checksum = "b05d47426645aef1e73b1a034c75ab2401bc504175feb191accbe211ec24a342" dependencies = [ "ahash", "arrow", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-execution 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-macros 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-physical-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "half", - "log", - "paste", -] - -[[package]] -name = "datafusion-functions-aggregate" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "half", "log", "paste", @@ -1978,21 +1865,9 @@ checksum = "05c99f648b2b1743de0c1c19eef07e8cc5a085237f172b2e20bf6934e0a804e4" dependencies = [ "ahash", "arrow", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - -[[package]] -name = "datafusion-functions-aggregate-common" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr-common", + "datafusion-physical-expr-common", ] [[package]] @@ -2003,36 +1878,15 @@ checksum = "4227782023f4fb68d3d5c5eb190665212f43c9a0b437553e4b938b379aff6cf6" dependencies = [ "arrow", "arrow-ord", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-doc 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-execution 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-functions 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-functions-aggregate 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-macros 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "itertools 0.14.0", - "log", - "paste", -] - -[[package]] -name = "datafusion-functions-nested" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "arrow", - "arrow-ord", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", + "datafusion-macros", + "datafusion-physical-expr-common", "itertools 0.14.0", "log", "paste", @@ -2040,14 +1894,15 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8ad370763644d6626b15900fe2268e7d55c618fadf5cff3a7f717bb6fb50ec1" dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr", "datafusion-physical-plan", "parking_lot", "paste", @@ -2055,17 +1910,18 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44b14fc52c77461f359d1697826a4373c7887a6adfca94eedc81c35decd0df9f" dependencies = [ "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-doc 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-window-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-doc", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "log", "paste", ] @@ -2076,17 +1932,8 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a1e149d36cdd44fb425dc815c5fac55025aa9a592dd65cb3c421881096292c02" dependencies = [ - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - -[[package]] -name = "datafusion-functions-window-common" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-physical-expr-common", ] [[package]] @@ -2095,32 +1942,23 @@ version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "07c9faa0cdefb6e6e756482b846397b5c2d84d369e30b009472b9ab9b1430fbd" dependencies = [ - "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "quote", - "syn 2.0.106", -] - -[[package]] -name = "datafusion-macros" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr", "quote", "syn 2.0.106", ] [[package]] name = "datafusion-optimizer" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b20ff1cec8c23fbab8523e2937790fb374b92d3b273306a64b7d8889ff3b8614" dependencies = [ "arrow", "chrono", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", "indexmap", "itertools 0.14.0", "log", @@ -2136,33 +1974,11 @@ checksum = "10bb87a605d8ce9672d5347c0293c12211b0c03923fc12fbdc665fe76e6f9e01" dependencies = [ "ahash", "arrow", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-functions-aggregate-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-physical-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "half", - "hashbrown 0.14.5", - "indexmap", - "itertools 0.14.0", - "log", - "parking_lot", - "paste", - "petgraph 0.8.2", -] - -[[package]] -name = "datafusion-physical-expr" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", "indexmap", @@ -2176,14 +1992,15 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-adapter" version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2da3a7429a555dd5ff0bec4d24bd5532ec43876764088da635cad55b2f178dc2" dependencies = [ "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr", + "datafusion-functions", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "itertools 0.14.0", ] @@ -2195,37 +2012,25 @@ checksum = "845eb44ef1e04d2a15c6d955cb146b40a41814a7be4377f0a541857d3e257d6f" dependencies = [ "ahash", "arrow", - "datafusion-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "datafusion-expr-common 50.0.0 (registry+https://github.com/rust-lang/crates.io-index)", - "hashbrown 0.14.5", - "itertools 0.14.0", -] - -[[package]] -name = "datafusion-physical-expr-common" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr-common", "hashbrown 0.14.5", "itertools 0.14.0", ] [[package]] name = "datafusion-physical-optimizer" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f96a93ebfd35cc52595e85c3100730a5baa6def39ff5390d6f90d2f3f89ce53f" dependencies = [ "arrow", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-pruning", "itertools 0.14.0", @@ -2234,8 +2039,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f6516a95911f763f05ec29bddd6fe987a0aa987409c213eac12faa5db7f3c9c" dependencies = [ "ahash", "arrow", @@ -2243,14 +2049,14 @@ dependencies = [ "arrow-schema", "async-trait", "chrono", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-aggregate-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions-window-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "futures", "half", "hashbrown 0.14.5", @@ -2264,16 +2070,17 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40befe63ab3bd9f3b05d02d13466055aa81876ad580247b10bdde1ba3782cebb" dependencies = [ "arrow", "arrow-schema", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-datasource", - "datafusion-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "itertools 0.14.0", "log", @@ -2281,17 +2088,18 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26aa059f478e6fa31158e80e4685226490b39f67c2e357401e26da84914be8b2" dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", "datafusion-common-runtime", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-physical-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-plan", "datafusion-sql", "futures", @@ -2304,18 +2112,19 @@ dependencies = [ [[package]] name = "datafusion-spark" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c11689184beea6390a30da81b1098487027bc9c8fcaa779a943c12c91520ac4" dependencies = [ "arrow", "chrono", "crc32fast", "datafusion-catalog", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-execution 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-functions 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-macros 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-macros", "log", "sha1", "url", @@ -2324,13 +2133,14 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "50.0.0" -source = "git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5#720b55a24918ac60a04dd377d7d9ab594741e7c5" +version = "50.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea3ce7cb3c31bfc6162026f6f4b11eb5a3a83c8a6b88d8b9c529ddbe97d53525" dependencies = [ "arrow", "bigdecimal", - "datafusion-common 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", - "datafusion-expr 50.0.0 (git+https://github.com/andygrove/datafusion?rev=720b55a24918ac60a04dd377d7d9ab594741e7c5)", + "datafusion-common", + "datafusion-expr", "indexmap", "log", "regex", diff --git a/native/Cargo.toml b/native/Cargo.toml index bf89c389bb..65b9a7eaf2 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -38,8 +38,8 @@ arrow = { version = "56.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "=56.0.0", default-features = false, features = ["experimental"] } -datafusion = { git = "https://github.com/andygrove/datafusion", rev = "720b55a24918ac60a04dd377d7d9ab594741e7c5", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } -datafusion-spark = { git = "https://github.com/andygrove/datafusion", rev = "720b55a24918ac60a04dd377d7d9ab594741e7c5" } +datafusion = { version = "50.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion-spark = { version = "50.0.0" } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } From 13f14d36939960bd973474c642b2a1df52c66f92 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 09:36:53 -0600 Subject: [PATCH 06/37] cargo update --- native/Cargo.lock | 273 ++++++++++++++++++++++------------------------ 1 file changed, 133 insertions(+), 140 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 11d3de3f08..78c238fd65 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -4,9 +4,9 @@ version = 4 [[package]] name = "addr2line" -version = "0.24.2" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" +checksum = "1b5d307320b3181d6d7954e663bd7c774a838b8220fe0593c86d9fb09f498b4b" dependencies = [ "gimli", ] @@ -87,9 +87,9 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstyle" -version = "1.0.11" +version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" +checksum = "5192cca8006f1fd4f7237516f40fa183bb07f8fbdfedaa0036de5ea9b0b45e78" [[package]] name = "anyhow" @@ -404,7 +404,7 @@ dependencies = [ "polling", "rustix 1.1.2", "slab", - "windows-sys 0.61.0", + "windows-sys 0.61.1", ] [[package]] @@ -484,9 +484,9 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.8.6" +version = "1.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bc1b40fb26027769f16960d2f4a6bc20c4bb755d403e552c8c1a73af433c246" +checksum = "04b37ddf8d2e9744a0b9c19ce0b78efe4795339a90b66b7bae77987092cd2e69" dependencies = [ "aws-credential-types", "aws-runtime", @@ -514,9 +514,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.6" +version = "1.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d025db5d9f52cbc413b167136afb3d8aeea708c0d8884783cf6253be5e22f6f2" +checksum = "799a1290207254984cb7c05245111bc77958b92a3c9bb449598044b36341cce6" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -526,9 +526,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.14.0" +version = "1.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94b8ff6c09cd57b16da53641caa860168b88c172a5ee163b0288d3d6eea12786" +checksum = "879b6c89592deb404ba4dc0ae6b58ffd1795c78991cbb5b8bc441c48a070440d" dependencies = [ "aws-lc-sys", "zeroize", @@ -536,22 +536,23 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.31.0" +version = "0.32.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e44d16778acaf6a9ec9899b92cebd65580b83f685446bf2e1f5d3d732f99dcd" +checksum = "a2b715a6010afb9e457ca2b7c9d2b9c344baa8baed7b38dc476034c171b32575" dependencies = [ "bindgen 0.72.1", "cc", "cmake", "dunce", "fs_extra", + "libloading 0.8.8", ] [[package]] name = "aws-runtime" -version = "1.5.10" +version = "1.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c034a1bc1d70e16e7f4e4caf7e9f7693e4c9c24cd91cf17c2a0b21abaebc7c8b" +checksum = "2e1ed337dabcf765ad5f2fb426f13af22d576328aaf09eac8f70953530798ec0" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -573,9 +574,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.84.0" +version = "1.85.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "357a841807f6b52cb26123878b3326921e2a25faca412fabdd32bd35b7edd5d3" +checksum = "2f2c741e2e439f07b5d1b33155e246742353d82167c785a2ff547275b7e32483" dependencies = [ "aws-credential-types", "aws-runtime", @@ -595,9 +596,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.85.0" +version = "1.87.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67e05f33b6c9026fecfe9b3b6740f34d41bc6ff641a6a32dabaab60209245b75" +checksum = "6428ae5686b18c0ee99f6f3c39d94ae3f8b42894cdc35c35d8fb2470e9db2d4c" dependencies = [ "aws-credential-types", "aws-runtime", @@ -617,9 +618,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.86.0" +version = "1.87.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7d835f123f307cafffca7b9027c14979f1d403b417d8541d67cf252e8a21e35" +checksum = "5871bec9a79a3e8d928c7788d654f135dde0e71d2dd98089388bab36b37ef607" dependencies = [ "aws-credential-types", "aws-runtime", @@ -693,9 +694,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.1" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "147e8eea63a40315d704b97bf9bc9b8c1402ae94f89d5ad6f7550d963309da1b" +checksum = "734b4282fbb7372923ac339cc2222530f8180d9d4745e582de19a18cee409fd8" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -843,9 +844,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.75" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" +checksum = "bb531853791a215d7c62a30daf0dde835f381ab5de4589cfe7c649d2cbe92bd6" dependencies = [ "addr2line", "cfg-if", @@ -853,7 +854,7 @@ dependencies = [ "miniz_oxide", "object", "rustc-demangle", - "windows-targets 0.52.6", + "windows-link", ] [[package]] @@ -1046,9 +1047,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.39" +version = "1.2.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1354349954c6fc9cb0deab020f27f783cf0b604e8bb754dc4658ecf0d29c35f" +checksum = "e1d05d92f4b1fd76aad469d46cdd858ca761576082cd37df81416691e50199fb" dependencies = [ "find-msvc-tools", "jobserver", @@ -1092,7 +1093,7 @@ dependencies = [ "iana-time-zone", "num-traits", "serde", - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -1140,7 +1141,7 @@ checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" dependencies = [ "glob", "libc", - "libloading 0.8.9", + "libloading 0.8.8", ] [[package]] @@ -1251,9 +1252,9 @@ checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" [[package]] name = "cpp_demangle" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96e58d342ad113c2b878f16d5d034c03be492ae460cdbc02b7f0f2284d310c7d" +checksum = "f2bb79cb74d735044c972aae58ed0aaa9a837e85b01106a54c39e42e97f62253" dependencies = [ "cfg-if", ] @@ -1600,9 +1601,9 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "054873d5563f115f83ef4270b560ac2ce4de713905e825a40cac49d6ff348254" +checksum = "eabb89b9d1ea8198d174b0838b91b40293b780261d694d6ac59bd20c38005115" dependencies = [ "ahash", "arrow", @@ -1749,15 +1750,15 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fbf41013cf55c2369b5229594898e8108c8a1beeb49d97feb5e0cce9933eb8f" +checksum = "b71f8c2c0d5c57620003c3bf1ee577b738404a7fd9642f6cf73d10e44ffaa70f" [[package]] name = "datafusion-execution" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26fd0c1ffe3885687758f985ed548184bf63b17b2a7a5ae695de422ad6432118" +checksum = "aa51cf4d253927cb65690c05a18e7720cdda4c47c923b0dd7d641f7fcfe21b14" dependencies = [ "arrow", "async-trait", @@ -1775,9 +1776,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c4fe6411218a9dab656437b1e69b00a470a7a2d7db087867a366c145eb164a7" +checksum = "4a347435cfcd1de0498c8410d32e0b1fc3920e198ce0378f8e259da717af9e0f" dependencies = [ "arrow", "async-trait", @@ -1796,9 +1797,9 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a45bee7d2606bfb41ceb1d904ba7cecf69bd5a6f8f3e6c57c3f5a83d84bdd97" +checksum = "4e73951bdf1047d7af212bb11310407230b4067921df648781ae7f7f1241e87e" dependencies = [ "arrow", "datafusion-common", @@ -1809,9 +1810,9 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c7e1c532ff9d14f291160bca23e55ffd4899800301dd2389786c2f02d76904a" +checksum = "a3b181e79552d764a2589910d1e0420ef41b07ab97c3e3efdbce612b692141e7" dependencies = [ "arrow", "arrow-buffer", @@ -1838,9 +1839,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b05d47426645aef1e73b1a034c75ab2401bc504175feb191accbe211ec24a342" +checksum = "b7e8cfb3b3f9e48e756939c85816b388264bed378d166a993fb265d800e1c83c" dependencies = [ "ahash", "arrow", @@ -1859,9 +1860,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05c99f648b2b1743de0c1c19eef07e8cc5a085237f172b2e20bf6934e0a804e4" +checksum = "9501537e235e4e86828bc8bf4e22968c1514c2cb4c860b7c7cf7dc99e172d43c" dependencies = [ "ahash", "arrow", @@ -1872,9 +1873,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4227782023f4fb68d3d5c5eb190665212f43c9a0b437553e4b938b379aff6cf6" +checksum = "6cbc3ecce122389530af091444e923f2f19153c38731893f5b798e19a46fbf86" dependencies = [ "arrow", "arrow-ord", @@ -1928,9 +1929,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1e149d36cdd44fb425dc815c5fac55025aa9a592dd65cb3c421881096292c02" +checksum = "851c80de71ff8bc9be7f8478f26e8060e25cab868a36190c4ebdaacc72ceade1" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1938,9 +1939,9 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07c9faa0cdefb6e6e756482b846397b5c2d84d369e30b009472b9ab9b1430fbd" +checksum = "386208ac4f475a099920cdbe9599188062276a09cb4c3f02efdc54e0c015ab14" dependencies = [ "datafusion-expr", "quote", @@ -1968,9 +1969,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10bb87a605d8ce9672d5347c0293c12211b0c03923fc12fbdc665fe76e6f9e01" +checksum = "945659046d27372e38e8a37927f0b887f50846202792063ad6b197c6eaf9fb5b" dependencies = [ "ahash", "arrow", @@ -1986,7 +1987,7 @@ dependencies = [ "log", "parking_lot", "paste", - "petgraph 0.8.2", + "petgraph 0.8.3", ] [[package]] @@ -2006,9 +2007,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "50.0.0" +version = "50.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "845eb44ef1e04d2a15c6d955cb146b40a41814a7be4377f0a541857d3e257d6f" +checksum = "218d60e94d829d8a52bf50e694f2f567313508f0c684af4954def9f774ce3518" dependencies = [ "ahash", "arrow", @@ -2158,9 +2159,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.3" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d630bccd429a5bb5a64b5e94f693bfc48c9f8566418fda4c494cc94f911f87cc" +checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" dependencies = [ "powerfmt", ] @@ -2259,7 +2260,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.0", + "windows-sys 0.61.1", ] [[package]] @@ -2297,9 +2298,9 @@ checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "find-msvc-tools" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ced73b1dacfc750a6db6c0a0c3a3853c8b41997e2e2c563dc90804ae6867959" +checksum = "0399f9d26e5191ce32c498bebd31e7a3ceabc2745f0ac54af3f335126c3f24b3" [[package]] name = "findshlibs" @@ -2321,9 +2322,9 @@ checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" [[package]] name = "flatbuffers" -version = "25.2.10" +version = "25.9.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" +checksum = "09b6620799e7340ebd9968d2e0708eb82cf1971e9a16821e2091b6d6e475eed5" dependencies = [ "bitflags 2.9.4", "rustc_version", @@ -2536,9 +2537,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.31.1" +version = "0.32.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" +checksum = "e629b9b98ef3dd8afe6ca2bd0f89306cec16d43d907889945bc5d6687f2f13c7" [[package]] name = "glob" @@ -2604,8 +2605,6 @@ version = "0.15.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" dependencies = [ - "allocator-api2", - "equivalent", "foldhash", ] @@ -3079,9 +3078,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.80" +version = "0.3.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "852f13bec5eba4ba9afbeb93fd7c13fe56147f055939ae21c43a29a0ecb2702e" +checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" dependencies = [ "once_cell", "wasm-bindgen", @@ -3183,12 +3182,12 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.9" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7c4b02199fee7c5d21a5ae7d8cfa79a6ef5bb2fc834d6e9058e89c825efdc55" +checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" dependencies = [ "cfg-if", - "windows-link 0.2.0", + "windows-targets 0.53.4", ] [[package]] @@ -3316,9 +3315,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.5" +version = "2.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" +checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" [[package]] name = "memmap2" @@ -3489,9 +3488,9 @@ dependencies = [ [[package]] name = "object" -version = "0.36.7" +version = "0.37.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" +checksum = "ff76201f031d8863c38aa7f905eca4f53abbfa15f609db4277d44cd8938f33fe" dependencies = [ "memchr", ] @@ -3705,9 +3704,9 @@ dependencies = [ [[package]] name = "petgraph" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54acf3a685220b533e437e264e4d932cfbdc4cc7ec0cd232ed73c08d03b8a7ca" +checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" dependencies = [ "fixedbitset", "hashbrown 0.15.5", @@ -3821,7 +3820,7 @@ dependencies = [ "hermit-abi", "pin-project-lite", "rustix 1.1.2", - "windows-sys 0.61.0", + "windows-sys 0.61.1", ] [[package]] @@ -4051,9 +4050,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.40" +version = "1.0.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" +checksum = "ce25767e7b499d1b604768e7cde645d14cc8584231ea6b295e9c9eb22c02e1d1" dependencies = [ "proc-macro2", ] @@ -4273,7 +4272,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.61.0", + "windows-sys 0.61.1", ] [[package]] @@ -4324,9 +4323,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.6" +version = "0.103.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8572f3c2cb9934231157b45499fc41e1f58c589fdfb81a844ba873265e80f8eb" +checksum = "e10b3f4191e8a80e6b43eebabfac91e5dcecebb27a71f04e820c47ec41d314bf" dependencies = [ "aws-lc-rs", "ring", @@ -4361,7 +4360,7 @@ version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "891d81b926048e76efe18581bf793546b4c0eaf8448d72be8de2bbee5fd166e1" dependencies = [ - "windows-sys 0.61.0", + "windows-sys 0.61.1", ] [[package]] @@ -4372,9 +4371,9 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "security-framework" -version = "3.5.0" +version = "3.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc198e42d9b7510827939c9a15f5062a0c913f3371d765977e586d2fe6c16f4a" +checksum = "b3297343eaf830f66ede390ea39da1d462b6b0c1b000f420d0a83f898bbbe6ef" dependencies = [ "bitflags 2.9.4", "core-foundation", @@ -4407,9 +4406,9 @@ checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" [[package]] name = "serde" -version = "1.0.226" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0dca6411025b24b60bfa7ec1fe1f8e710ac09782dca409ee8237ba74b51295fd" +checksum = "9a8e94ea7f378bd32cbbd37198a4a91436180c5bb472411e48b5ec2e2124ae9e" dependencies = [ "serde_core", "serde_derive", @@ -4427,18 +4426,18 @@ dependencies = [ [[package]] name = "serde_core" -version = "1.0.226" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba2ba63999edb9dac981fb34b3e5c0d111a69b0924e253ed29d83f7c99e966a4" +checksum = "41d385c7d4ca58e59fc732af25c3983b67ac852c1a25000afe1175de458b67ad" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.226" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8db53ae22f34573731bafa1db20f04027b2d25e02d8205921b569171699cdb33" +checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", @@ -4708,7 +4707,7 @@ dependencies = [ "getrandom 0.3.3", "once_cell", "rustix 1.1.2", - "windows-sys 0.61.0", + "windows-sys 0.61.1", ] [[package]] @@ -4910,9 +4909,9 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.3" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05f63835928ca123f1bef57abbcd23bb2ba0ac9ae1235f1e65bda0d06e7786bd" +checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" dependencies = [ "rustls", "tokio", @@ -5033,9 +5032,9 @@ dependencies = [ [[package]] name = "typenum" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" +checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "unicode-ident" @@ -5181,9 +5180,9 @@ dependencies = [ [[package]] name = "wasm-bindgen" -version = "0.2.103" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab10a69fbd0a177f5f649ad4d8d3305499c42bab9aef2f7ff592d0ec8f833819" +checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" dependencies = [ "cfg-if", "once_cell", @@ -5194,9 +5193,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.103" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bb702423545a6007bbc368fde243ba47ca275e549c8a28617f56f6ba53b1d1c" +checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" dependencies = [ "bumpalo", "log", @@ -5208,9 +5207,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.53" +version = "0.4.54" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0b221ff421256839509adbb55998214a70d829d3a28c69b4a6672e9d2a42f67" +checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" dependencies = [ "cfg-if", "js-sys", @@ -5221,9 +5220,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.103" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc65f4f411d91494355917b605e1480033152658d71f722a90647f56a70c88a0" +checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -5231,9 +5230,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.103" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffc003a991398a8ee604a401e194b6b3a39677b3173d6e74495eb51b82e99a32" +checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" dependencies = [ "proc-macro2", "quote", @@ -5244,9 +5243,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.103" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "293c37f4efa430ca14db3721dfbe48d8c33308096bd44d80ebaa775ab71ba1cf" +checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" dependencies = [ "unicode-ident", ] @@ -5266,9 +5265,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.80" +version = "0.3.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbe734895e869dc429d78c4b433f8d17d95f8d05317440b4fad5ab2d33e596dc" +checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" dependencies = [ "js-sys", "wasm-bindgen", @@ -5327,7 +5326,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.0", + "windows-sys 0.61.1", ] [[package]] @@ -5338,22 +5337,22 @@ checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] name = "windows-core" -version = "0.62.0" +version = "0.62.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57fe7168f7de578d2d8a05b07fd61870d2e73b4020e9f49aa00da8471723497c" +checksum = "6844ee5416b285084d3d3fffd743b925a6c9385455f64f6d4fa3031c4c2749a9" dependencies = [ "windows-implement", "windows-interface", - "windows-link 0.2.0", + "windows-link", "windows-result", "windows-strings", ] [[package]] name = "windows-implement" -version = "0.60.0" +version = "0.60.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" +checksum = "edb307e42a74fb6de9bf3a02d9712678b22399c87e6fa869d6dfcd8c1b7754e0" dependencies = [ "proc-macro2", "quote", @@ -5362,21 +5361,15 @@ dependencies = [ [[package]] name = "windows-interface" -version = "0.59.1" +version = "0.59.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" +checksum = "c0abd1ddbc6964ac14db11c7213d6532ef34bd9aa042c2e5935f59d7908b46a5" dependencies = [ "proc-macro2", "quote", "syn 2.0.106", ] -[[package]] -name = "windows-link" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" - [[package]] name = "windows-link" version = "0.2.0" @@ -5389,7 +5382,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7084dcc306f89883455a206237404d3eaf961e5bd7e0f312f7c91f57eb44167f" dependencies = [ - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -5398,7 +5391,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7218c655a553b0bed4426cf54b20d7ba363ef543b52d515b3e48d7fd55318dda" dependencies = [ - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -5434,16 +5427,16 @@ version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" dependencies = [ - "windows-targets 0.53.3", + "windows-targets 0.53.4", ] [[package]] name = "windows-sys" -version = "0.61.0" +version = "0.61.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e201184e40b2ede64bc2ea34968b28e33622acdbbf37104f0e4a33f7abe657aa" +checksum = "6f109e41dd4a3c848907eb83d5a42ea98b3769495597450cf6d153507b166f0f" dependencies = [ - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -5479,11 +5472,11 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.53.3" +version = "0.53.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" +checksum = "2d42b7b7f66d2a06854650af09cfdf8713e427a439c97ad65a6375318033ac4b" dependencies = [ - "windows-link 0.1.3", + "windows-link", "windows_aarch64_gnullvm 0.53.0", "windows_aarch64_msvc 0.53.0", "windows_i686_gnu 0.53.0", @@ -5723,9 +5716,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" +checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" From 78f5b4f04d1bbf5241a775c4271a896c542d2d8c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 09:37:43 -0600 Subject: [PATCH 07/37] [skip ci] From dc11515d53f2ea5770ad51383e9343a054ddd0a3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 09:47:02 -0600 Subject: [PATCH 08/37] save [skip-ci] --- .../src/main/scala/org/apache/comet/CometConf.scala | 13 +++++++++++-- native/core/src/execution/jni_api.rs | 12 +++++++----- native/core/src/execution/spark_config.rs | 2 ++ .../scala/org/apache/comet/CometExecIterator.scala | 4 ++-- 4 files changed, 22 insertions(+), 9 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index c2a5d05829..6211e18c53 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -63,9 +63,11 @@ object CometConf extends ShimCometConf { def conf(key: String): ConfigBuilder = ConfigBuilder(key) - val COMET_EXEC_CONFIG_PREFIX = "spark.comet.exec"; + val COMET_PREFIX = "spark.comet"; - val COMET_EXPR_CONFIG_PREFIX = "spark.comet.expression"; + val COMET_EXEC_CONFIG_PREFIX = s"$COMET_PREFIX.exec"; + + val COMET_EXPR_CONFIG_PREFIX = s"$COMET_PREFIX.expression"; val COMET_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.enabled") .doc( @@ -454,6 +456,13 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_DEBUG_MEMORY_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.debug.memory") + .doc("TBD") + .internal() + .booleanConf + .createWithDefault(false) + val COMET_EXPLAIN_VERBOSE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.explain.verbose.enabled") .doc( diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index a8bc30524f..1d572dc07c 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -79,9 +79,7 @@ use crate::execution::spark_plan::SparkPlan; use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace}; use crate::execution::memory_pools::logging_pool::LoggingPool; -use crate::execution::spark_config::{ - SparkConfig, COMET_DEBUG_ENABLED, COMET_EXPLAIN_NATIVE_ENABLED, COMET_TRACING_ENABLED, -}; +use crate::execution::spark_config::{SparkConfig, COMET_DEBUG_ENABLED, COMET_DEBUG_MEMORY, COMET_EXPLAIN_NATIVE_ENABLED, COMET_TRACING_ENABLED}; use datafusion_comet_proto::spark_operator::operator::OpStruct; use log::info; use once_cell::sync::Lazy; @@ -183,6 +181,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED); let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED); let tracing_enabled = spark_config.get_bool(COMET_TRACING_ENABLED); + let logging_memory_pool = spark_config.get_bool(COMET_DEBUG_MEMORY); with_trace("createPlan", tracing_enabled, || { // Init JVM classes @@ -221,8 +220,11 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let memory_pool = create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id); - // TODO only if debug option set - let memory_pool = Arc::new(LoggingPool::new(memory_pool)); + let memory_pool = if logging_memory_pool { + Arc::new(LoggingPool::new(memory_pool)) + } else { + memory_pool + }; // Get local directories for storing spill files let local_dirs_array = JObjectArray::from_raw(local_dirs); diff --git a/native/core/src/execution/spark_config.rs b/native/core/src/execution/spark_config.rs index 7465a1ea9f..5bf8aaf3a9 100644 --- a/native/core/src/execution/spark_config.rs +++ b/native/core/src/execution/spark_config.rs @@ -21,6 +21,8 @@ pub(crate) const COMET_TRACING_ENABLED: &str = "spark.comet.tracing.enabled"; pub(crate) const COMET_DEBUG_ENABLED: &str = "spark.comet.debug.enabled"; pub(crate) const COMET_EXPLAIN_NATIVE_ENABLED: &str = "spark.comet.explain.native.enabled"; +pub(crate) const COMET_DEBUG_MEMORY: &str = "spark.comet.debug.memory"; + pub(crate) trait SparkConfig { fn get_bool(&self, name: &str) -> bool; } diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 9b2e131488..95e45ecae3 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -87,9 +87,9 @@ class CometExecIterator( CometSparkSessionExtensions.getCometMemoryOverhead(conf) } - // serialize Spark conf in protobuf format + // serialize Comet configs in protobuf format val builder = ConfigMap.newBuilder() - conf.getAll.foreach { case (k, v) => + conf.getAll.filter(_._1.startsWith(CometConf.COMET_PREFIX)).foreach { case (k, v) => builder.putEntries(k, v) } val protobufSparkConfigs = builder.build().toByteArray From d2a1ab1c02add9ab899fa16eb5a2767f2ce3081c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 09:51:52 -0600 Subject: [PATCH 09/37] [skip ci] From 31cdbc6a5b0af81b437603d9875d8010e288e000 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 10:17:50 -0600 Subject: [PATCH 10/37] save [skip ci] --- dev/benchmarks/comet-tpch.sh | 7 ++++--- native/core/src/errors.rs | 8 ++------ native/core/src/execution/jni_api.rs | 6 ++++++ 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index b895333185..435b89c031 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -30,11 +30,11 @@ RUST_BACKTRACE=1 $SPARK_HOME/bin/spark-submit \ --driver-class-path $COMET_JAR \ --conf spark.driver.memory=8G \ --conf spark.executor.instances=1 \ - --conf spark.executor.cores=1 \ - --conf spark.cores.max=1 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=8 \ --conf spark.executor.memory=16g \ --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=1g \ + --conf spark.memory.offHeap.size=2g \ --conf spark.eventLog.enabled=true \ --conf spark.driver.extraClassPath=$COMET_JAR \ --conf spark.executor.extraClassPath=$COMET_JAR \ @@ -42,6 +42,7 @@ RUST_BACKTRACE=1 $SPARK_HOME/bin/spark-submit \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ --conf spark.comet.exec.replaceSortMergeJoin=false \ --conf spark.comet.expression.allowIncompatible=true \ + --conf spark.comet.debug.memory=true \ --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ tpcbench.py \ diff --git a/native/core/src/errors.rs b/native/core/src/errors.rs index 32f05c3c8e..b3241477b8 100644 --- a/native/core/src/errors.rs +++ b/native/core/src/errors.rs @@ -42,7 +42,6 @@ use datafusion_comet_spark_expr::SparkError; use jni::objects::{GlobalRef, JThrowable, JValue}; use jni::JNIEnv; use lazy_static::lazy_static; -use log::error; use parquet::errors::ParquetError; use thiserror::Error; @@ -147,11 +146,9 @@ pub enum CometError { pub fn init() { std::panic::set_hook(Box::new(|_panic_info| { - let x = std::backtrace::Backtrace::force_capture().to_string(); - error!("PANIC: {x}"); - // Capture the backtrace for a panic - *PANIC_BACKTRACE.lock().unwrap() = Some(x); + *PANIC_BACKTRACE.lock().unwrap() = + Some(std::backtrace::Backtrace::force_capture().to_string()); })); } @@ -362,7 +359,6 @@ pub fn unwrap_or_throw_default( CometError::Panic { msg: _ } => PANIC_BACKTRACE.lock().unwrap().take(), _ => None, }; - error!("Comet native call failed: {err:?} {backtrace:?}"); throw_exception(env, &err, backtrace); T::default() } diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 1d572dc07c..aaadb50078 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -177,12 +177,18 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let spark_configs = serde::deserialize_config(bytes.as_slice())?; let spark_config: HashMap = spark_configs.entries.into_iter().collect(); + for (k, v) in &spark_config { + println!("CONF: {k} = {v}"); + } + // Access Spark configs let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED); let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED); let tracing_enabled = spark_config.get_bool(COMET_TRACING_ENABLED); let logging_memory_pool = spark_config.get_bool(COMET_DEBUG_MEMORY); + println!("logging_memory_pool = {logging_memory_pool}"); + with_trace("createPlan", tracing_enabled, || { // Init JVM classes JVMClasses::init(&mut env); From 322b4c5d6c96f7a83bf1147fe13e0b4e68ec6937 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 10:23:08 -0600 Subject: [PATCH 11/37] info logging --- native/core/src/execution/jni_api.rs | 8 +------- native/core/src/execution/memory_pools/logging_pool.rs | 10 +++++----- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index aaadb50078..ae83c984f1 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -177,18 +177,12 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let spark_configs = serde::deserialize_config(bytes.as_slice())?; let spark_config: HashMap = spark_configs.entries.into_iter().collect(); - for (k, v) in &spark_config { - println!("CONF: {k} = {v}"); - } - - // Access Spark configs + // Access Comet configs let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED); let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED); let tracing_enabled = spark_config.get_bool(COMET_TRACING_ENABLED); let logging_memory_pool = spark_config.get_bool(COMET_DEBUG_MEMORY); - println!("logging_memory_pool = {logging_memory_pool}"); - with_trace("createPlan", tracing_enabled, || { // Init JVM classes JVMClasses::init(&mut env); diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index cb10f39792..bcd9cedc9a 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -16,7 +16,7 @@ // under the License. use datafusion::execution::memory_pool::{MemoryPool, MemoryReservation}; -use log::trace; +use log::info; use std::sync::Arc; #[derive(Debug)] @@ -32,7 +32,7 @@ impl LoggingPool { impl MemoryPool for LoggingPool { fn grow(&self, reservation: &MemoryReservation, additional: usize) { - trace!( + info!( "MemoryPool[{}].grow({})", reservation.consumer().name(), reservation.size() @@ -41,7 +41,7 @@ impl MemoryPool for LoggingPool { } fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { - trace!( + info!( "MemoryPool[{}].shrink({})", reservation.consumer().name(), reservation.size() @@ -56,13 +56,13 @@ impl MemoryPool for LoggingPool { ) -> datafusion::common::Result<()> { let result = self.pool.try_grow(reservation, additional); if result.is_ok() { - trace!( + info!( "MemoryPool[{}].try_grow({}) returning Ok", reservation.consumer().name(), reservation.size() ); } else { - trace!( + info!( "MemoryPool[{}].try_grow({}) returning Err", reservation.consumer().name(), reservation.size() From 89e10acd48e61a3ee0f95bf0cddb9c9b192511f0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 10:53:28 -0600 Subject: [PATCH 12/37] log task id [skip ci] --- .../scala/org/apache/comet/CometConf.scala | 4 ++-- native/core/src/execution/jni_api.rs | 7 +++++-- .../execution/memory_pools/logging_pool.rs | 20 +++++++++++++------ 3 files changed, 21 insertions(+), 10 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 6211e18c53..a928f551a3 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -65,9 +65,9 @@ object CometConf extends ShimCometConf { val COMET_PREFIX = "spark.comet"; - val COMET_EXEC_CONFIG_PREFIX = s"$COMET_PREFIX.exec"; + val COMET_EXEC_CONFIG_PREFIX: String = s"$COMET_PREFIX.exec"; - val COMET_EXPR_CONFIG_PREFIX = s"$COMET_PREFIX.expression"; + val COMET_EXPR_CONFIG_PREFIX: String = s"$COMET_PREFIX.expression"; val COMET_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.enabled") .doc( diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index ae83c984f1..0d0034fe3b 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -79,7 +79,10 @@ use crate::execution::spark_plan::SparkPlan; use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace}; use crate::execution::memory_pools::logging_pool::LoggingPool; -use crate::execution::spark_config::{SparkConfig, COMET_DEBUG_ENABLED, COMET_DEBUG_MEMORY, COMET_EXPLAIN_NATIVE_ENABLED, COMET_TRACING_ENABLED}; +use crate::execution::spark_config::{ + SparkConfig, COMET_DEBUG_ENABLED, COMET_DEBUG_MEMORY, COMET_EXPLAIN_NATIVE_ENABLED, + COMET_TRACING_ENABLED, +}; use datafusion_comet_proto::spark_operator::operator::OpStruct; use log::info; use once_cell::sync::Lazy; @@ -221,7 +224,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id); let memory_pool = if logging_memory_pool { - Arc::new(LoggingPool::new(memory_pool)) + Arc::new(LoggingPool::new(task_attempt_id as u64, memory_pool)) } else { memory_pool }; diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index bcd9cedc9a..3e78484df6 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -21,19 +21,24 @@ use std::sync::Arc; #[derive(Debug)] pub(crate) struct LoggingPool { + task_attempt_id: u64, pool: Arc, } impl LoggingPool { - pub fn new(pool: Arc) -> Self { - Self { pool } + pub fn new(task_attempt_id: u64, pool: Arc) -> Self { + Self { + task_attempt_id, + pool, + } } } impl MemoryPool for LoggingPool { fn grow(&self, reservation: &MemoryReservation, additional: usize) { info!( - "MemoryPool[{}].grow({})", + "[Task {}] MemoryPool[{}].grow({})", + self.task_attempt_id, reservation.consumer().name(), reservation.size() ); @@ -42,7 +47,8 @@ impl MemoryPool for LoggingPool { fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { info!( - "MemoryPool[{}].shrink({})", + "[Task {}] MemoryPool[{}].shrink({})", + self.task_attempt_id, reservation.consumer().name(), reservation.size() ); @@ -57,13 +63,15 @@ impl MemoryPool for LoggingPool { let result = self.pool.try_grow(reservation, additional); if result.is_ok() { info!( - "MemoryPool[{}].try_grow({}) returning Ok", + "[Task {}] MemoryPool[{}].try_grow({}) returning Ok", + self.task_attempt_id, reservation.consumer().name(), reservation.size() ); } else { info!( - "MemoryPool[{}].try_grow({}) returning Err", + "[Task {}] MemoryPool[{}].try_grow({}) returning Err", + self.task_attempt_id, reservation.consumer().name(), reservation.size() ); From 3b191fd6dde85dc88ce34e1ae14d2d26d760b003 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 11:32:57 -0600 Subject: [PATCH 13/37] println --- native/core/src/execution/memory_pools/logging_pool.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index 3e78484df6..391096423a 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -16,7 +16,6 @@ // under the License. use datafusion::execution::memory_pool::{MemoryPool, MemoryReservation}; -use log::info; use std::sync::Arc; #[derive(Debug)] @@ -36,7 +35,7 @@ impl LoggingPool { impl MemoryPool for LoggingPool { fn grow(&self, reservation: &MemoryReservation, additional: usize) { - info!( + println!( "[Task {}] MemoryPool[{}].grow({})", self.task_attempt_id, reservation.consumer().name(), @@ -46,7 +45,7 @@ impl MemoryPool for LoggingPool { } fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { - info!( + println!( "[Task {}] MemoryPool[{}].shrink({})", self.task_attempt_id, reservation.consumer().name(), @@ -62,14 +61,14 @@ impl MemoryPool for LoggingPool { ) -> datafusion::common::Result<()> { let result = self.pool.try_grow(reservation, additional); if result.is_ok() { - info!( + println!( "[Task {}] MemoryPool[{}].try_grow({}) returning Ok", self.task_attempt_id, reservation.consumer().name(), reservation.size() ); } else { - info!( + println!( "[Task {}] MemoryPool[{}].try_grow({}) returning Err", self.task_attempt_id, reservation.consumer().name(), From 7c24836581df46413789cb25d6257ac91de1ba68 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 11:33:22 -0600 Subject: [PATCH 14/37] revert lock file --- native/Cargo.lock | 341 +++++++++++++++++++++++----------------------- 1 file changed, 174 insertions(+), 167 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 78c238fd65..c707a72537 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -4,9 +4,9 @@ version = 4 [[package]] name = "addr2line" -version = "0.25.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b5d307320b3181d6d7954e663bd7c774a838b8220fe0593c86d9fb09f498b4b" +checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" dependencies = [ "gimli", ] @@ -87,9 +87,9 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstyle" -version = "1.0.13" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5192cca8006f1fd4f7237516f40fa183bb07f8fbdfedaa0036de5ea9b0b45e78" +checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" [[package]] name = "anyhow" @@ -404,7 +404,7 @@ dependencies = [ "polling", "rustix 1.1.2", "slab", - "windows-sys 0.61.1", + "windows-sys 0.61.0", ] [[package]] @@ -484,9 +484,9 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.8.7" +version = "1.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04b37ddf8d2e9744a0b9c19ce0b78efe4795339a90b66b7bae77987092cd2e69" +checksum = "8bc1b40fb26027769f16960d2f4a6bc20c4bb755d403e552c8c1a73af433c246" dependencies = [ "aws-credential-types", "aws-runtime", @@ -514,9 +514,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.7" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "799a1290207254984cb7c05245111bc77958b92a3c9bb449598044b36341cce6" +checksum = "d025db5d9f52cbc413b167136afb3d8aeea708c0d8884783cf6253be5e22f6f2" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -526,9 +526,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.14.1" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879b6c89592deb404ba4dc0ae6b58ffd1795c78991cbb5b8bc441c48a070440d" +checksum = "94b8ff6c09cd57b16da53641caa860168b88c172a5ee163b0288d3d6eea12786" dependencies = [ "aws-lc-sys", "zeroize", @@ -536,23 +536,22 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.32.2" +version = "0.31.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2b715a6010afb9e457ca2b7c9d2b9c344baa8baed7b38dc476034c171b32575" +checksum = "0e44d16778acaf6a9ec9899b92cebd65580b83f685446bf2e1f5d3d732f99dcd" dependencies = [ "bindgen 0.72.1", "cc", "cmake", "dunce", "fs_extra", - "libloading 0.8.8", ] [[package]] name = "aws-runtime" -version = "1.5.11" +version = "1.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e1ed337dabcf765ad5f2fb426f13af22d576328aaf09eac8f70953530798ec0" +checksum = "c034a1bc1d70e16e7f4e4caf7e9f7693e4c9c24cd91cf17c2a0b21abaebc7c8b" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -574,9 +573,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.85.0" +version = "1.84.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f2c741e2e439f07b5d1b33155e246742353d82167c785a2ff547275b7e32483" +checksum = "357a841807f6b52cb26123878b3326921e2a25faca412fabdd32bd35b7edd5d3" dependencies = [ "aws-credential-types", "aws-runtime", @@ -596,9 +595,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.87.0" +version = "1.85.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6428ae5686b18c0ee99f6f3c39d94ae3f8b42894cdc35c35d8fb2470e9db2d4c" +checksum = "67e05f33b6c9026fecfe9b3b6740f34d41bc6ff641a6a32dabaab60209245b75" dependencies = [ "aws-credential-types", "aws-runtime", @@ -618,9 +617,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.87.0" +version = "1.86.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5871bec9a79a3e8d928c7788d654f135dde0e71d2dd98089388bab36b37ef607" +checksum = "e7d835f123f307cafffca7b9027c14979f1d403b417d8541d67cf252e8a21e35" dependencies = [ "aws-credential-types", "aws-runtime", @@ -694,9 +693,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.2" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "734b4282fbb7372923ac339cc2222530f8180d9d4745e582de19a18cee409fd8" +checksum = "147e8eea63a40315d704b97bf9bc9b8c1402ae94f89d5ad6f7550d963309da1b" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -844,9 +843,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.76" +version = "0.3.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb531853791a215d7c62a30daf0dde835f381ab5de4589cfe7c649d2cbe92bd6" +checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" dependencies = [ "addr2line", "cfg-if", @@ -854,7 +853,7 @@ dependencies = [ "miniz_oxide", "object", "rustc-demangle", - "windows-link", + "windows-targets 0.52.6", ] [[package]] @@ -1047,9 +1046,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.40" +version = "1.2.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1d05d92f4b1fd76aad469d46cdd858ca761576082cd37df81416691e50199fb" +checksum = "e1354349954c6fc9cb0deab020f27f783cf0b604e8bb754dc4658ecf0d29c35f" dependencies = [ "find-msvc-tools", "jobserver", @@ -1093,7 +1092,7 @@ dependencies = [ "iana-time-zone", "num-traits", "serde", - "windows-link", + "windows-link 0.2.0", ] [[package]] @@ -1141,7 +1140,7 @@ checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" dependencies = [ "glob", "libc", - "libloading 0.8.8", + "libloading 0.8.9", ] [[package]] @@ -1252,9 +1251,9 @@ checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" [[package]] name = "cpp_demangle" -version = "0.4.5" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2bb79cb74d735044c972aae58ed0aaa9a837e85b01106a54c39e42e97f62253" +checksum = "96e58d342ad113c2b878f16d5d034c03be492ae460cdbc02b7f0f2284d310c7d" dependencies = [ "cfg-if", ] @@ -1390,9 +1389,9 @@ dependencies = [ [[package]] name = "datafusion" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4016a135c11820d9c9884a1f7924d5456c563bd3657b7d691a6e7b937a452df7" +checksum = "481d0c1cad7606cee11233abcdff8eec46e43dd25abda007db6d5d26ae8483c4" dependencies = [ "arrow", "arrow-ipc", @@ -1441,9 +1440,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1721d3973afeb8a0c3f235a79101cc61e4a558dd3f02fdc9ae6c61e882e544d9" +checksum = "d70327e81ab3a1f5832d8b372d55fa607851d7cea6d1f8e65ff0c98fcc32d222" dependencies = [ "arrow", "async-trait", @@ -1467,9 +1466,9 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44841d3efb0c89c6a5ac6fde5ac61d4f2474a2767f170db6d97300a8b4df8904" +checksum = "268819e6bb20ba70a664abddc20deac604f30d3267f8c91847064542a8c0720c" dependencies = [ "arrow", "async-trait", @@ -1601,9 +1600,9 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eabb89b9d1ea8198d174b0838b91b40293b780261d694d6ac59bd20c38005115" +checksum = "054873d5563f115f83ef4270b560ac2ce4de713905e825a40cac49d6ff348254" dependencies = [ "ahash", "arrow", @@ -1625,9 +1624,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f03fe3936f978fe8e76776d14ad8722e33843b01d81d11707ca72d54d2867787" +checksum = "b8a1d1bc69aaaadb8008b65329ed890b33e845dc063225c190f77b20328fbe1d" dependencies = [ "futures", "log", @@ -1636,9 +1635,9 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4543216d2f4fc255780a46ae9e062e50c86ac23ecab6718cc1ba3fe4a8d5a8f2" +checksum = "d855160469020982880fd9bd0962e033d2f4728f56f85a83d8c90785638b6519" dependencies = [ "arrow", "async-trait", @@ -1667,9 +1666,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ab662d4692ca5929ce32eb609c6c8a741772537d98363b3efb3bc68148cd530" +checksum = "9ec3aa7575378d23aae96b955b5233bea6f9d461648174f6ccc8f3c160f2b7a7" dependencies = [ "arrow", "async-trait", @@ -1692,9 +1691,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dad4492ba9a2fca417cb211f8f05ffeb7f12a1f0f8e5bdcf548c353ff923779" +checksum = "00cfb8f33e2864eeb3188b6818acf5546d56a5a487d423cce9b684a554caabfa" dependencies = [ "arrow", "async-trait", @@ -1717,9 +1716,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2925432ce04847cc09b4789a53fc22b0fdf5f2e73289ad7432759d76c6026e9e" +checksum = "ab3bfb48fb4ff42ac1485a12ea56434eaab53f7da8f00b2443b1a3d35a0b6d10" dependencies = [ "arrow", "async-trait", @@ -1750,15 +1749,15 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b71f8c2c0d5c57620003c3bf1ee577b738404a7fd9642f6cf73d10e44ffaa70f" +checksum = "2fbf41013cf55c2369b5229594898e8108c8a1beeb49d97feb5e0cce9933eb8f" [[package]] name = "datafusion-execution" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa51cf4d253927cb65690c05a18e7720cdda4c47c923b0dd7d641f7fcfe21b14" +checksum = "26fd0c1ffe3885687758f985ed548184bf63b17b2a7a5ae695de422ad6432118" dependencies = [ "arrow", "async-trait", @@ -1776,9 +1775,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a347435cfcd1de0498c8410d32e0b1fc3920e198ce0378f8e259da717af9e0f" +checksum = "5c4fe6411218a9dab656437b1e69b00a470a7a2d7db087867a366c145eb164a7" dependencies = [ "arrow", "async-trait", @@ -1797,9 +1796,9 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e73951bdf1047d7af212bb11310407230b4067921df648781ae7f7f1241e87e" +checksum = "4a45bee7d2606bfb41ceb1d904ba7cecf69bd5a6f8f3e6c57c3f5a83d84bdd97" dependencies = [ "arrow", "datafusion-common", @@ -1810,9 +1809,9 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3b181e79552d764a2589910d1e0420ef41b07ab97c3e3efdbce612b692141e7" +checksum = "9c7e1c532ff9d14f291160bca23e55ffd4899800301dd2389786c2f02d76904a" dependencies = [ "arrow", "arrow-buffer", @@ -1839,9 +1838,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b7e8cfb3b3f9e48e756939c85816b388264bed378d166a993fb265d800e1c83c" +checksum = "b05d47426645aef1e73b1a034c75ab2401bc504175feb191accbe211ec24a342" dependencies = [ "ahash", "arrow", @@ -1860,9 +1859,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9501537e235e4e86828bc8bf4e22968c1514c2cb4c860b7c7cf7dc99e172d43c" +checksum = "05c99f648b2b1743de0c1c19eef07e8cc5a085237f172b2e20bf6934e0a804e4" dependencies = [ "ahash", "arrow", @@ -1873,9 +1872,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6cbc3ecce122389530af091444e923f2f19153c38731893f5b798e19a46fbf86" +checksum = "4227782023f4fb68d3d5c5eb190665212f43c9a0b437553e4b938b379aff6cf6" dependencies = [ "arrow", "arrow-ord", @@ -1895,9 +1894,9 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8ad370763644d6626b15900fe2268e7d55c618fadf5cff3a7f717bb6fb50ec1" +checksum = "3d902b1769f69058236e89f04f3bff2cf62f24311adb7bf3c6c3e945c9451076" dependencies = [ "arrow", "async-trait", @@ -1911,9 +1910,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44b14fc52c77461f359d1697826a4373c7887a6adfca94eedc81c35decd0df9f" +checksum = "4b8ee43974c92eb9920fe8e97e0fab48675e93b062abcb48bef4c1d4305b6ee4" dependencies = [ "arrow", "datafusion-common", @@ -1929,9 +1928,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "851c80de71ff8bc9be7f8478f26e8060e25cab868a36190c4ebdaacc72ceade1" +checksum = "a1e149d36cdd44fb425dc815c5fac55025aa9a592dd65cb3c421881096292c02" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1939,9 +1938,9 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "386208ac4f475a099920cdbe9599188062276a09cb4c3f02efdc54e0c015ab14" +checksum = "07c9faa0cdefb6e6e756482b846397b5c2d84d369e30b009472b9ab9b1430fbd" dependencies = [ "datafusion-expr", "quote", @@ -1950,9 +1949,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b20ff1cec8c23fbab8523e2937790fb374b92d3b273306a64b7d8889ff3b8614" +checksum = "f16a4f7059302ad1de6e97ab0eebb5c34405917b1f80806a30a66e38ad118251" dependencies = [ "arrow", "chrono", @@ -1969,9 +1968,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "945659046d27372e38e8a37927f0b887f50846202792063ad6b197c6eaf9fb5b" +checksum = "10bb87a605d8ce9672d5347c0293c12211b0c03923fc12fbdc665fe76e6f9e01" dependencies = [ "ahash", "arrow", @@ -1987,7 +1986,7 @@ dependencies = [ "log", "parking_lot", "paste", - "petgraph 0.8.3", + "petgraph 0.8.2", ] [[package]] @@ -2007,9 +2006,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "218d60e94d829d8a52bf50e694f2f567313508f0c684af4954def9f774ce3518" +checksum = "845eb44ef1e04d2a15c6d955cb146b40a41814a7be4377f0a541857d3e257d6f" dependencies = [ "ahash", "arrow", @@ -2021,9 +2020,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f96a93ebfd35cc52595e85c3100730a5baa6def39ff5390d6f90d2f3f89ce53f" +checksum = "32b9b648ee2785722c79eae366528e52e93ece6808aef9297cf8e5521de381da" dependencies = [ "arrow", "datafusion-common", @@ -2040,9 +2039,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f6516a95911f763f05ec29bddd6fe987a0aa987409c213eac12faa5db7f3c9c" +checksum = "7e6688d17b78104e169d7069749832c20ff50f112be853d2c058afe46c889064" dependencies = [ "ahash", "arrow", @@ -2071,9 +2070,9 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40befe63ab3bd9f3b05d02d13466055aa81876ad580247b10bdde1ba3782cebb" +checksum = "8a893a46c56f5f190085e13949eb8ec163672c7ec2ac33bdb82c84572e71ca73" dependencies = [ "arrow", "arrow-schema", @@ -2089,9 +2088,9 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26aa059f478e6fa31158e80e4685226490b39f67c2e357401e26da84914be8b2" +checksum = "f8b62684c7a1db6121a8c83100209cffa1e664a8d9ced87e1a32f8cdc2fff3c2" dependencies = [ "arrow", "async-trait", @@ -2113,9 +2112,9 @@ dependencies = [ [[package]] name = "datafusion-spark" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c11689184beea6390a30da81b1098487027bc9c8fcaa779a943c12c91520ac4" +checksum = "b217679a4f1bbfae8ec97143950c42940eebba7c566b4141184fa8f81d296db1" dependencies = [ "arrow", "chrono", @@ -2134,9 +2133,9 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "50.1.0" +version = "50.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea3ce7cb3c31bfc6162026f6f4b11eb5a3a83c8a6b88d8b9c529ddbe97d53525" +checksum = "f09cff94b8242843e1da5d069e9d2cfc53807f1f00b1c0da78c297f47c21456e" dependencies = [ "arrow", "bigdecimal", @@ -2159,9 +2158,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.4" +version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" +checksum = "d630bccd429a5bb5a64b5e94f693bfc48c9f8566418fda4c494cc94f911f87cc" dependencies = [ "powerfmt", ] @@ -2260,7 +2259,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.1", + "windows-sys 0.61.0", ] [[package]] @@ -2298,9 +2297,9 @@ checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "find-msvc-tools" -version = "0.1.3" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0399f9d26e5191ce32c498bebd31e7a3ceabc2745f0ac54af3f335126c3f24b3" +checksum = "1ced73b1dacfc750a6db6c0a0c3a3853c8b41997e2e2c563dc90804ae6867959" [[package]] name = "findshlibs" @@ -2322,9 +2321,9 @@ checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" [[package]] name = "flatbuffers" -version = "25.9.23" +version = "25.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09b6620799e7340ebd9968d2e0708eb82cf1971e9a16821e2091b6d6e475eed5" +checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" dependencies = [ "bitflags 2.9.4", "rustc_version", @@ -2537,9 +2536,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.32.3" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e629b9b98ef3dd8afe6ca2bd0f89306cec16d43d907889945bc5d6687f2f13c7" +checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" @@ -2605,6 +2604,8 @@ version = "0.15.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" dependencies = [ + "allocator-api2", + "equivalent", "foldhash", ] @@ -3078,9 +3079,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.81" +version = "0.3.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" +checksum = "852f13bec5eba4ba9afbeb93fd7c13fe56147f055939ae21c43a29a0ecb2702e" dependencies = [ "once_cell", "wasm-bindgen", @@ -3182,12 +3183,12 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" +checksum = "d7c4b02199fee7c5d21a5ae7d8cfa79a6ef5bb2fc834d6e9058e89c825efdc55" dependencies = [ "cfg-if", - "windows-targets 0.53.4", + "windows-link 0.2.0", ] [[package]] @@ -3315,9 +3316,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.6" +version = "2.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" +checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" [[package]] name = "memmap2" @@ -3488,9 +3489,9 @@ dependencies = [ [[package]] name = "object" -version = "0.37.3" +version = "0.36.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff76201f031d8863c38aa7f905eca4f53abbfa15f609db4277d44cd8938f33fe" +checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" dependencies = [ "memchr", ] @@ -3704,9 +3705,9 @@ dependencies = [ [[package]] name = "petgraph" -version = "0.8.3" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" +checksum = "54acf3a685220b533e437e264e4d932cfbdc4cc7ec0cd232ed73c08d03b8a7ca" dependencies = [ "fixedbitset", "hashbrown 0.15.5", @@ -3820,7 +3821,7 @@ dependencies = [ "hermit-abi", "pin-project-lite", "rustix 1.1.2", - "windows-sys 0.61.1", + "windows-sys 0.61.0", ] [[package]] @@ -4050,9 +4051,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.41" +version = "1.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce25767e7b499d1b604768e7cde645d14cc8584231ea6b295e9c9eb22c02e1d1" +checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" dependencies = [ "proc-macro2", ] @@ -4272,7 +4273,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.11.0", - "windows-sys 0.61.1", + "windows-sys 0.61.0", ] [[package]] @@ -4323,9 +4324,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.7" +version = "0.103.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e10b3f4191e8a80e6b43eebabfac91e5dcecebb27a71f04e820c47ec41d314bf" +checksum = "8572f3c2cb9934231157b45499fc41e1f58c589fdfb81a844ba873265e80f8eb" dependencies = [ "aws-lc-rs", "ring", @@ -4360,7 +4361,7 @@ version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "891d81b926048e76efe18581bf793546b4c0eaf8448d72be8de2bbee5fd166e1" dependencies = [ - "windows-sys 0.61.1", + "windows-sys 0.61.0", ] [[package]] @@ -4371,9 +4372,9 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "security-framework" -version = "3.5.1" +version = "3.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3297343eaf830f66ede390ea39da1d462b6b0c1b000f420d0a83f898bbbe6ef" +checksum = "cc198e42d9b7510827939c9a15f5062a0c913f3371d765977e586d2fe6c16f4a" dependencies = [ "bitflags 2.9.4", "core-foundation", @@ -4406,9 +4407,9 @@ checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" [[package]] name = "serde" -version = "1.0.228" +version = "1.0.226" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a8e94ea7f378bd32cbbd37198a4a91436180c5bb472411e48b5ec2e2124ae9e" +checksum = "0dca6411025b24b60bfa7ec1fe1f8e710ac09782dca409ee8237ba74b51295fd" dependencies = [ "serde_core", "serde_derive", @@ -4426,18 +4427,18 @@ dependencies = [ [[package]] name = "serde_core" -version = "1.0.228" +version = "1.0.226" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41d385c7d4ca58e59fc732af25c3983b67ac852c1a25000afe1175de458b67ad" +checksum = "ba2ba63999edb9dac981fb34b3e5c0d111a69b0924e253ed29d83f7c99e966a4" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.228" +version = "1.0.226" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" +checksum = "8db53ae22f34573731bafa1db20f04027b2d25e02d8205921b569171699cdb33" dependencies = [ "proc-macro2", "quote", @@ -4707,7 +4708,7 @@ dependencies = [ "getrandom 0.3.3", "once_cell", "rustix 1.1.2", - "windows-sys 0.61.1", + "windows-sys 0.61.0", ] [[package]] @@ -4909,9 +4910,9 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.4" +version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" +checksum = "05f63835928ca123f1bef57abbcd23bb2ba0ac9ae1235f1e65bda0d06e7786bd" dependencies = [ "rustls", "tokio", @@ -5032,9 +5033,9 @@ dependencies = [ [[package]] name = "typenum" -version = "1.19.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" +checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" [[package]] name = "unicode-ident" @@ -5180,9 +5181,9 @@ dependencies = [ [[package]] name = "wasm-bindgen" -version = "0.2.104" +version = "0.2.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" +checksum = "ab10a69fbd0a177f5f649ad4d8d3305499c42bab9aef2f7ff592d0ec8f833819" dependencies = [ "cfg-if", "once_cell", @@ -5193,9 +5194,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.104" +version = "0.2.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" +checksum = "0bb702423545a6007bbc368fde243ba47ca275e549c8a28617f56f6ba53b1d1c" dependencies = [ "bumpalo", "log", @@ -5207,9 +5208,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.54" +version = "0.4.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" +checksum = "a0b221ff421256839509adbb55998214a70d829d3a28c69b4a6672e9d2a42f67" dependencies = [ "cfg-if", "js-sys", @@ -5220,9 +5221,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.104" +version = "0.2.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" +checksum = "fc65f4f411d91494355917b605e1480033152658d71f722a90647f56a70c88a0" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -5230,9 +5231,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.104" +version = "0.2.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" +checksum = "ffc003a991398a8ee604a401e194b6b3a39677b3173d6e74495eb51b82e99a32" dependencies = [ "proc-macro2", "quote", @@ -5243,9 +5244,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.104" +version = "0.2.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" +checksum = "293c37f4efa430ca14db3721dfbe48d8c33308096bd44d80ebaa775ab71ba1cf" dependencies = [ "unicode-ident", ] @@ -5265,9 +5266,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.81" +version = "0.3.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" +checksum = "fbe734895e869dc429d78c4b433f8d17d95f8d05317440b4fad5ab2d33e596dc" dependencies = [ "js-sys", "wasm-bindgen", @@ -5326,7 +5327,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.1", + "windows-sys 0.61.0", ] [[package]] @@ -5337,22 +5338,22 @@ checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] name = "windows-core" -version = "0.62.1" +version = "0.62.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6844ee5416b285084d3d3fffd743b925a6c9385455f64f6d4fa3031c4c2749a9" +checksum = "57fe7168f7de578d2d8a05b07fd61870d2e73b4020e9f49aa00da8471723497c" dependencies = [ "windows-implement", "windows-interface", - "windows-link", + "windows-link 0.2.0", "windows-result", "windows-strings", ] [[package]] name = "windows-implement" -version = "0.60.1" +version = "0.60.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edb307e42a74fb6de9bf3a02d9712678b22399c87e6fa869d6dfcd8c1b7754e0" +checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" dependencies = [ "proc-macro2", "quote", @@ -5361,15 +5362,21 @@ dependencies = [ [[package]] name = "windows-interface" -version = "0.59.2" +version = "0.59.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0abd1ddbc6964ac14db11c7213d6532ef34bd9aa042c2e5935f59d7908b46a5" +checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" dependencies = [ "proc-macro2", "quote", "syn 2.0.106", ] +[[package]] +name = "windows-link" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" + [[package]] name = "windows-link" version = "0.2.0" @@ -5382,7 +5389,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7084dcc306f89883455a206237404d3eaf961e5bd7e0f312f7c91f57eb44167f" dependencies = [ - "windows-link", + "windows-link 0.2.0", ] [[package]] @@ -5391,7 +5398,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7218c655a553b0bed4426cf54b20d7ba363ef543b52d515b3e48d7fd55318dda" dependencies = [ - "windows-link", + "windows-link 0.2.0", ] [[package]] @@ -5427,16 +5434,16 @@ version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" dependencies = [ - "windows-targets 0.53.4", + "windows-targets 0.53.3", ] [[package]] name = "windows-sys" -version = "0.61.1" +version = "0.61.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f109e41dd4a3c848907eb83d5a42ea98b3769495597450cf6d153507b166f0f" +checksum = "e201184e40b2ede64bc2ea34968b28e33622acdbbf37104f0e4a33f7abe657aa" dependencies = [ - "windows-link", + "windows-link 0.2.0", ] [[package]] @@ -5472,11 +5479,11 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.53.4" +version = "0.53.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d42b7b7f66d2a06854650af09cfdf8713e427a439c97ad65a6375318033ac4b" +checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" dependencies = [ - "windows-link", + "windows-link 0.1.3", "windows_aarch64_gnullvm 0.53.0", "windows_aarch64_msvc 0.53.0", "windows_i686_gnu 0.53.0", @@ -5716,9 +5723,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.8.2" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" +checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" [[package]] name = "zerotrie" From 405f5b7b1fc51e1f7fe633b0ece83cd0da34d41b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 11:34:42 -0600 Subject: [PATCH 15/37] prep for review --- common/src/main/scala/org/apache/comet/CometConf.scala | 2 +- dev/benchmarks/comet-tpch.sh | 9 ++++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index a928f551a3..8cead1bafd 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -458,7 +458,7 @@ object CometConf extends ShimCometConf { val COMET_DEBUG_MEMORY_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.debug.memory") - .doc("TBD") + .doc("When enabled, log all native memory pool interactions to stdout.") .internal() .booleanConf .createWithDefault(false) diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index 435b89c031..e1f6f969ff 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -22,9 +22,9 @@ $SPARK_HOME/sbin/stop-master.sh $SPARK_HOME/sbin/stop-worker.sh $SPARK_HOME/sbin/start-master.sh -RUST_BACKTRACE=1 $SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER +$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER -RUST_BACKTRACE=1 $SPARK_HOME/bin/spark-submit \ +$SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ --jars $COMET_JAR \ --driver-class-path $COMET_JAR \ @@ -34,15 +34,14 @@ RUST_BACKTRACE=1 $SPARK_HOME/bin/spark-submit \ --conf spark.cores.max=8 \ --conf spark.executor.memory=16g \ --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=2g \ + --conf spark.memory.offHeap.size=16g \ --conf spark.eventLog.enabled=true \ --conf spark.driver.extraClassPath=$COMET_JAR \ --conf spark.executor.extraClassPath=$COMET_JAR \ --conf spark.plugins=org.apache.spark.CometPlugin \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.exec.replaceSortMergeJoin=false \ + --conf spark.comet.exec.replaceSortMergeJoin=true \ --conf spark.comet.expression.allowIncompatible=true \ - --conf spark.comet.debug.memory=true \ --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ tpcbench.py \ From 522238d2dfbf72a69dc4901e86031a772367ba37 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 11:39:18 -0600 Subject: [PATCH 16/37] save --- common/src/main/scala/org/apache/comet/CometConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 8cead1bafd..3fa962d59b 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -457,7 +457,7 @@ object CometConf extends ShimCometConf { .createWithDefault(false) val COMET_DEBUG_MEMORY_ENABLED: ConfigEntry[Boolean] = - conf("spark.comet.debug.memory") + conf(s"$COMET_PREFIX.debug.memory") .doc("When enabled, log all native memory pool interactions to stdout.") .internal() .booleanConf From 36565ca4f629792198f518ccd556a053e15fb584 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 12:18:41 -0600 Subject: [PATCH 17/37] Update spark/src/main/scala/org/apache/comet/CometExecIterator.scala Co-authored-by: Oleks V --- spark/src/main/scala/org/apache/comet/CometExecIterator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 95e45ecae3..aea35b2069 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -87,7 +87,7 @@ class CometExecIterator( CometSparkSessionExtensions.getCometMemoryOverhead(conf) } - // serialize Comet configs in protobuf format + // serialize Comet related Spark configs in protobuf format val builder = ConfigMap.newBuilder() conf.getAll.filter(_._1.startsWith(CometConf.COMET_PREFIX)).foreach { case (k, v) => builder.putEntries(k, v) From 21189a6915be90c7c6355f8a0613b26fbaf52417 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 12:18:57 -0600 Subject: [PATCH 18/37] info logging --- native/core/src/execution/memory_pools/logging_pool.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index 391096423a..3e78484df6 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -16,6 +16,7 @@ // under the License. use datafusion::execution::memory_pool::{MemoryPool, MemoryReservation}; +use log::info; use std::sync::Arc; #[derive(Debug)] @@ -35,7 +36,7 @@ impl LoggingPool { impl MemoryPool for LoggingPool { fn grow(&self, reservation: &MemoryReservation, additional: usize) { - println!( + info!( "[Task {}] MemoryPool[{}].grow({})", self.task_attempt_id, reservation.consumer().name(), @@ -45,7 +46,7 @@ impl MemoryPool for LoggingPool { } fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { - println!( + info!( "[Task {}] MemoryPool[{}].shrink({})", self.task_attempt_id, reservation.consumer().name(), @@ -61,14 +62,14 @@ impl MemoryPool for LoggingPool { ) -> datafusion::common::Result<()> { let result = self.pool.try_grow(reservation, additional); if result.is_ok() { - println!( + info!( "[Task {}] MemoryPool[{}].try_grow({}) returning Ok", self.task_attempt_id, reservation.consumer().name(), reservation.size() ); } else { - println!( + info!( "[Task {}] MemoryPool[{}].try_grow({}) returning Err", self.task_attempt_id, reservation.consumer().name(), From d9817ce84c570bbe7f907eb8f430e258fe165088 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 13:28:20 -0600 Subject: [PATCH 19/37] fix --- native/core/src/execution/memory_pools/logging_pool.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index 3e78484df6..11649270a9 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -40,7 +40,7 @@ impl MemoryPool for LoggingPool { "[Task {}] MemoryPool[{}].grow({})", self.task_attempt_id, reservation.consumer().name(), - reservation.size() + additional ); self.pool.grow(reservation, additional); } @@ -50,7 +50,7 @@ impl MemoryPool for LoggingPool { "[Task {}] MemoryPool[{}].shrink({})", self.task_attempt_id, reservation.consumer().name(), - reservation.size() + shrink ); self.pool.shrink(reservation, shrink); } @@ -66,14 +66,14 @@ impl MemoryPool for LoggingPool { "[Task {}] MemoryPool[{}].try_grow({}) returning Ok", self.task_attempt_id, reservation.consumer().name(), - reservation.size() + additional ); } else { info!( "[Task {}] MemoryPool[{}].try_grow({}) returning Err", self.task_attempt_id, reservation.consumer().name(), - reservation.size() + additional ); } result From acba7bc5cd74fd4cae0ca8d3fd048cf78676fde4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 14:17:23 -0600 Subject: [PATCH 20/37] log error on try_grow fail --- dev/benchmarks/comet-tpch.sh | 5 +-- .../execution/memory_pools/logging_pool.rs | 35 ++++++++++--------- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index e1f6f969ff..66cf4ab01c 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -34,14 +34,15 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.cores.max=8 \ --conf spark.executor.memory=16g \ --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ + --conf spark.memory.offHeap.size=2g \ --conf spark.eventLog.enabled=true \ --conf spark.driver.extraClassPath=$COMET_JAR \ --conf spark.executor.extraClassPath=$COMET_JAR \ --conf spark.plugins=org.apache.spark.CometPlugin \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.exec.replaceSortMergeJoin=true \ + --conf spark.comet.exec.replaceSortMergeJoin=false \ --conf spark.comet.expression.allowIncompatible=true \ + --conf spark.comet.debug.memory=true \ --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ tpcbench.py \ diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index 11649270a9..f2e47c5b50 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -60,23 +60,26 @@ impl MemoryPool for LoggingPool { reservation: &MemoryReservation, additional: usize, ) -> datafusion::common::Result<()> { - let result = self.pool.try_grow(reservation, additional); - if result.is_ok() { - info!( - "[Task {}] MemoryPool[{}].try_grow({}) returning Ok", - self.task_attempt_id, - reservation.consumer().name(), - additional - ); - } else { - info!( - "[Task {}] MemoryPool[{}].try_grow({}) returning Err", - self.task_attempt_id, - reservation.consumer().name(), - additional - ); + match self.pool.try_grow(reservation, additional) { + Ok(_) => { + info!( + "[Task {}] MemoryPool[{}].try_grow({}) returning Ok", + self.task_attempt_id, + reservation.consumer().name(), + additional + ); + Ok(()) + } + Err(e) => { + info!( + "[Task {}] MemoryPool[{}].try_grow({}) returning Err: {:?}", + self.task_attempt_id, + reservation.consumer().name(), + additional + ); + Err(e) + } } - result } fn reserved(&self) -> usize { From 4051d290c4996ce0898a0ee73d3d2adc7be405aa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 14:17:54 -0600 Subject: [PATCH 21/37] log error on try_grow fail --- native/core/src/execution/memory_pools/logging_pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index f2e47c5b50..af37782b7d 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -72,7 +72,7 @@ impl MemoryPool for LoggingPool { } Err(e) => { info!( - "[Task {}] MemoryPool[{}].try_grow({}) returning Err: {:?}", + "[Task {}] MemoryPool[{}].try_grow({}) returning Err: {e:?}", self.task_attempt_id, reservation.consumer().name(), additional From df69875a2153924cf47c2540aef85c43a076b56c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 14:18:28 -0600 Subject: [PATCH 22/37] revert --- dev/benchmarks/comet-tpch.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index 66cf4ab01c..e1f6f969ff 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -34,15 +34,14 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.cores.max=8 \ --conf spark.executor.memory=16g \ --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=2g \ + --conf spark.memory.offHeap.size=16g \ --conf spark.eventLog.enabled=true \ --conf spark.driver.extraClassPath=$COMET_JAR \ --conf spark.executor.extraClassPath=$COMET_JAR \ --conf spark.plugins=org.apache.spark.CometPlugin \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.exec.replaceSortMergeJoin=false \ + --conf spark.comet.exec.replaceSortMergeJoin=true \ --conf spark.comet.expression.allowIncompatible=true \ - --conf spark.comet.debug.memory=true \ --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ tpcbench.py \ From ad891a09c2a26ea5886976a64afea320401f68f4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 14:44:33 -0600 Subject: [PATCH 23/37] add Python script to convert log to csv --- dev/scripts/mem_debug_to_csv.py | 69 +++++++++++++++++++++++++++++++++ 1 file changed, 69 insertions(+) create mode 100644 dev/scripts/mem_debug_to_csv.py diff --git a/dev/scripts/mem_debug_to_csv.py b/dev/scripts/mem_debug_to_csv.py new file mode 100644 index 0000000000..1bc4770cc6 --- /dev/null +++ b/dev/scripts/mem_debug_to_csv.py @@ -0,0 +1,69 @@ +#!/usr/bin/python +############################################################################## +# 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. +############################################################################## + +import argparse +import re +import sys + +def main(file, task_filter): + # keep track of running total allocation per consumer + alloc = {} + + # open file + with open(file) as f: + # iterate over lines in file + print("name,size") + for line in f: + # print(line, file=sys.stderr) + + # example line: [Task 486] MemoryPool[HashJoinInput[6]].shrink(1000) + # parse consumer name + re_match = re.search('\[Task (.*)\] MemoryPool\[(.*)\]\.(.*)\((.*)\)', line, re.IGNORECASE) + if re_match: + try: + task = int(re_match.group(1)) + if task != task_filter: + continue + + consumer = re_match.group(2) + method = re_match.group(3) + size = int(re_match.group(4)) + if method == "try_grow": + if "Err" in line: + continue + + if alloc.get(consumer) is None: + alloc[consumer] = size + else: + if method == "grow" or method == "try_grow": + alloc[consumer] = alloc[consumer] + size + elif method == "shrink": + alloc[consumer] = alloc[consumer] - size + print(consumer, ",", alloc[consumer]) + except: + print("error parsing", line, file=sys.stderr) + + +if __name__ == "__main__": + ap = argparse.ArgumentParser(description="Generate CSV From memory debug output") + ap.add_argument("--task", default=None, help="Task ID.") + ap.add_argument("--file", default=None, help="Spark log containing memory debug output") + args = ap.parse_args() + main(args.file, int(args.task)) From 87562562df0924609a1f9fc06c5c506ded9d89db Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 14:49:44 -0600 Subject: [PATCH 24/37] Python script to generate chart --- dev/scripts/mem_debug_chart.py | 80 ++++++++++++++++++++++++++++++++++ 1 file changed, 80 insertions(+) create mode 100644 dev/scripts/mem_debug_chart.py diff --git a/dev/scripts/mem_debug_chart.py b/dev/scripts/mem_debug_chart.py new file mode 100644 index 0000000000..9e77ebfcf8 --- /dev/null +++ b/dev/scripts/mem_debug_chart.py @@ -0,0 +1,80 @@ +#!/usr/bin/python +############################################################################## +# 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. +############################################################################## + +import argparse +from pathlib import Path +import pandas as pd +import matplotlib.pyplot as plt + +def main(): + ap = argparse.ArgumentParser() + ap.add_argument("csv", help="CSV with columns: name,size") + ap.add_argument("--instant", action="store_true", + help="Plot per-step stacked values (not cumulative totals).") + ap.add_argument("--bar", action="store_true", + help="Use stacked bars instead of stacked area.") + ap.add_argument("--title", default=None, help="Optional plot title.") + args = ap.parse_args() + + path = Path(args.csv) + df = pd.read_csv(path) + + # Validate + clean + need = {"name", "size"} + if not need.issubset(set(df.columns)): + raise SystemExit("CSV must have columns: name,size") + + df["size"] = pd.to_numeric(df["size"], errors="coerce").fillna(0) + + # Treat each row as the next time step: t = 1..N + df = df.reset_index(drop=True).assign(t=lambda d: d.index + 1) + + # Build wide matrix: one column per name, one row per time step + # If multiple entries exist for the same (t, name), they’ll be summed. + wide = ( + df.groupby(["t", "name"], as_index=False)["size"].sum() + .pivot(index="t", columns="name", values="size") + .fillna(0.0) + .sort_index() + ) + + # Running totals unless --instant specified + plot_data = wide if args.instant else wide.cumsum(axis=0) + + # Plot + if args.bar: + ax = plot_data.plot(kind="bar", stacked=True, figsize=(12, 6), width=1.0) + else: + ax = plot_data.plot.area(stacked=True, figsize=(12, 6)) + + ax.set_xlabel("step") + ax.set_ylabel("size" if args.instant else "cumulative size") + ax.set_title(args.title or ("Stacked running totals by name" if not args.instant + else "Stacked per-step values by name")) + ax.legend(title="name", bbox_to_anchor=(1.02, 1), loc="upper left") + plt.tight_layout() + + out = path.with_suffix(".stacked.png" if args.instant else ".stacked_cumulative.png") + plt.savefig(out, dpi=150) + print(f"Saved plot to {out}") + plt.show() + +if __name__ == "__main__": + main() From 7eb1bc17dcc33079b8198924a6359c008dba693e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 15:00:48 -0600 Subject: [PATCH 25/37] scripts --- dev/scripts/mem_debug_chart.py | 21 +++++++-------------- dev/scripts/mem_debug_to_csv.py | 8 +++++++- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/dev/scripts/mem_debug_chart.py b/dev/scripts/mem_debug_chart.py index 9e77ebfcf8..b613de77cd 100644 --- a/dev/scripts/mem_debug_chart.py +++ b/dev/scripts/mem_debug_chart.py @@ -26,10 +26,6 @@ def main(): ap = argparse.ArgumentParser() ap.add_argument("csv", help="CSV with columns: name,size") - ap.add_argument("--instant", action="store_true", - help="Plot per-step stacked values (not cumulative totals).") - ap.add_argument("--bar", action="store_true", - help="Use stacked bars instead of stacked area.") ap.add_argument("--title", default=None, help="Optional plot title.") args = ap.parse_args() @@ -55,23 +51,20 @@ def main(): .sort_index() ) - # Running totals unless --instant specified - plot_data = wide if args.instant else wide.cumsum(axis=0) + # Running totals + plot_data = wide + # plot_data = wide.cumsum(axis=0) # Plot - if args.bar: - ax = plot_data.plot(kind="bar", stacked=True, figsize=(12, 6), width=1.0) - else: - ax = plot_data.plot.area(stacked=True, figsize=(12, 6)) + ax = plot_data.plot.area(stacked=True, figsize=(12, 6)) ax.set_xlabel("step") - ax.set_ylabel("size" if args.instant else "cumulative size") - ax.set_title(args.title or ("Stacked running totals by name" if not args.instant - else "Stacked per-step values by name")) + ax.set_ylabel("cumulative size") + ax.set_title(args.title or ("Stacked running totals by name")) ax.legend(title="name", bbox_to_anchor=(1.02, 1), loc="upper left") plt.tight_layout() - out = path.with_suffix(".stacked.png" if args.instant else ".stacked_cumulative.png") + out = path.with_suffix(".stacked_cumulative.png") plt.savefig(out, dpi=150) print(f"Saved plot to {out}") plt.show() diff --git a/dev/scripts/mem_debug_to_csv.py b/dev/scripts/mem_debug_to_csv.py index 1bc4770cc6..c26f3c2994 100644 --- a/dev/scripts/mem_debug_to_csv.py +++ b/dev/scripts/mem_debug_to_csv.py @@ -56,7 +56,13 @@ def main(file, task_filter): alloc[consumer] = alloc[consumer] + size elif method == "shrink": alloc[consumer] = alloc[consumer] - size - print(consumer, ",", alloc[consumer]) + + if alloc[consumer] >= 0: + print(consumer, ",", alloc[consumer]) + else: + print("ignoring negative size ", consumer, ",", alloc[consumer], file=sys.stderr) + print(consumer, ",", 0) + except: print("error parsing", line, file=sys.stderr) From 21bd38685093ca164cb51f6238733d0cba373b42 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 15:12:44 -0600 Subject: [PATCH 26/37] new script --- dev/scripts/mem_debug_chart.py | 73 -------------------------------- dev/scripts/plot_memory_usage.py | 42 ++++++++++++++++++ 2 files changed, 42 insertions(+), 73 deletions(-) delete mode 100644 dev/scripts/mem_debug_chart.py create mode 100644 dev/scripts/plot_memory_usage.py diff --git a/dev/scripts/mem_debug_chart.py b/dev/scripts/mem_debug_chart.py deleted file mode 100644 index b613de77cd..0000000000 --- a/dev/scripts/mem_debug_chart.py +++ /dev/null @@ -1,73 +0,0 @@ -#!/usr/bin/python -############################################################################## -# 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. -############################################################################## - -import argparse -from pathlib import Path -import pandas as pd -import matplotlib.pyplot as plt - -def main(): - ap = argparse.ArgumentParser() - ap.add_argument("csv", help="CSV with columns: name,size") - ap.add_argument("--title", default=None, help="Optional plot title.") - args = ap.parse_args() - - path = Path(args.csv) - df = pd.read_csv(path) - - # Validate + clean - need = {"name", "size"} - if not need.issubset(set(df.columns)): - raise SystemExit("CSV must have columns: name,size") - - df["size"] = pd.to_numeric(df["size"], errors="coerce").fillna(0) - - # Treat each row as the next time step: t = 1..N - df = df.reset_index(drop=True).assign(t=lambda d: d.index + 1) - - # Build wide matrix: one column per name, one row per time step - # If multiple entries exist for the same (t, name), they’ll be summed. - wide = ( - df.groupby(["t", "name"], as_index=False)["size"].sum() - .pivot(index="t", columns="name", values="size") - .fillna(0.0) - .sort_index() - ) - - # Running totals - plot_data = wide - # plot_data = wide.cumsum(axis=0) - - # Plot - ax = plot_data.plot.area(stacked=True, figsize=(12, 6)) - - ax.set_xlabel("step") - ax.set_ylabel("cumulative size") - ax.set_title(args.title or ("Stacked running totals by name")) - ax.legend(title="name", bbox_to_anchor=(1.02, 1), loc="upper left") - plt.tight_layout() - - out = path.with_suffix(".stacked_cumulative.png") - plt.savefig(out, dpi=150) - print(f"Saved plot to {out}") - plt.show() - -if __name__ == "__main__": - main() diff --git a/dev/scripts/plot_memory_usage.py b/dev/scripts/plot_memory_usage.py new file mode 100644 index 0000000000..93fc898a08 --- /dev/null +++ b/dev/scripts/plot_memory_usage.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python3 +import pandas as pd +import matplotlib.pyplot as plt +import sys + +def plot_memory_usage(csv_file): + # Read the CSV file + df = pd.read_csv(csv_file) + + # Create time index based on row order (each row is a sequential time point) + df['time'] = range(len(df)) + + # Pivot the data to have consumers as columns + pivot_df = df.pivot(index='time', columns='name', values='size') + pivot_df = pivot_df.fillna(method='ffill').fillna(0) + + # Create stacked area chart + plt.figure(figsize=(12, 6)) + plt.stackplot(pivot_df.index, + [pivot_df[col] for col in pivot_df.columns], + labels=pivot_df.columns, + alpha=0.8) + + plt.xlabel('Time') + plt.ylabel('Memory Usage') + plt.title('Memory Usage Over Time by Consumer') + plt.legend(loc='upper left') + plt.grid(True, alpha=0.3) + plt.tight_layout() + + # Save the plot + output_file = csv_file.replace('.csv', '_chart.png') + plt.savefig(output_file, dpi=300, bbox_inches='tight') + print(f"Chart saved to: {output_file}") + plt.show() + +if __name__ == "__main__": + if len(sys.argv) != 2: + print("Usage: python plot_memory_usage.py ") + sys.exit(1) + + plot_memory_usage(sys.argv[1]) From ec823c2e5f7e6cb93de87f479e96856efd8200ae Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 15:24:48 -0600 Subject: [PATCH 27/37] show err --- dev/scripts/plot_memory_usage.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/dev/scripts/plot_memory_usage.py b/dev/scripts/plot_memory_usage.py index 93fc898a08..7c15753892 100644 --- a/dev/scripts/plot_memory_usage.py +++ b/dev/scripts/plot_memory_usage.py @@ -21,6 +21,14 @@ def plot_memory_usage(csv_file): labels=pivot_df.columns, alpha=0.8) + # Add annotations for ERR labels + if 'label' in df.columns: + err_points = df[df['label'].str.contains('ERR', na=False)] + for _, row in err_points.iterrows(): + plt.axvline(x=row['time'], color='red', linestyle='--', alpha=0.7, linewidth=1.5) + plt.text(row['time'], plt.ylim()[1] * 0.95, 'ERR', + ha='center', va='top', color='red', fontweight='bold') + plt.xlabel('Time') plt.ylabel('Memory Usage') plt.title('Memory Usage Over Time by Consumer') From a66fa65376b67b9fa29cdce8ac0d000682317f76 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 15:25:02 -0600 Subject: [PATCH 28/37] save --- dev/scripts/mem_debug_to_csv.py | 28 +++++++++++++--------------- dev/scripts/plot_memory_usage.py | 2 +- 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/dev/scripts/mem_debug_to_csv.py b/dev/scripts/mem_debug_to_csv.py index c26f3c2994..233ada287d 100644 --- a/dev/scripts/mem_debug_to_csv.py +++ b/dev/scripts/mem_debug_to_csv.py @@ -29,7 +29,7 @@ def main(file, task_filter): # open file with open(file) as f: # iterate over lines in file - print("name,size") + print("name,size,label") for line in f: # print(line, file=sys.stderr) @@ -45,23 +45,21 @@ def main(file, task_filter): consumer = re_match.group(2) method = re_match.group(3) size = int(re_match.group(4)) - if method == "try_grow": - if "Err" in line: - continue - if alloc.get(consumer) is None: - alloc[consumer] = size + if method == "try_grow" and "Err" in line: + # do not update allocation if try_grow failed + # annotate this entry so it can be shown in the chart + print(consumer, ",", alloc[consumer], ",ERR") + pass else: - if method == "grow" or method == "try_grow": - alloc[consumer] = alloc[consumer] + size - elif method == "shrink": - alloc[consumer] = alloc[consumer] - size - - if alloc[consumer] >= 0: + if alloc.get(consumer) is None: + alloc[consumer] = size + else: + if method == "grow" or method == "try_grow": + alloc[consumer] = alloc[consumer] + size + elif method == "shrink": + alloc[consumer] = alloc[consumer] - size print(consumer, ",", alloc[consumer]) - else: - print("ignoring negative size ", consumer, ",", alloc[consumer], file=sys.stderr) - print(consumer, ",", 0) except: print("error parsing", line, file=sys.stderr) diff --git a/dev/scripts/plot_memory_usage.py b/dev/scripts/plot_memory_usage.py index 93fc898a08..f972c673c1 100644 --- a/dev/scripts/plot_memory_usage.py +++ b/dev/scripts/plot_memory_usage.py @@ -15,7 +15,7 @@ def plot_memory_usage(csv_file): pivot_df = pivot_df.fillna(method='ffill').fillna(0) # Create stacked area chart - plt.figure(figsize=(12, 6)) + plt.figure(figsize=(8, 4)) plt.stackplot(pivot_df.index, [pivot_df[col] for col in pivot_df.columns], labels=pivot_df.columns, From 2fb336e0cddf09a35dd2c475b4ce6c800114976f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 15:48:13 -0600 Subject: [PATCH 29/37] track errors --- dev/scripts/mem_debug_to_csv.py | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/dev/scripts/mem_debug_to_csv.py b/dev/scripts/mem_debug_to_csv.py index 233ada287d..ad2c7af052 100644 --- a/dev/scripts/mem_debug_to_csv.py +++ b/dev/scripts/mem_debug_to_csv.py @@ -35,7 +35,7 @@ def main(file, task_filter): # example line: [Task 486] MemoryPool[HashJoinInput[6]].shrink(1000) # parse consumer name - re_match = re.search('\[Task (.*)\] MemoryPool\[(.*)\]\.(.*)\((.*)\)', line, re.IGNORECASE) + re_match = re.search('\[Task (.*)\] MemoryPool\[(.*)\]\.(try_grow|grow|shrink)\(([0-9]*)\)', line, re.IGNORECASE) if re_match: try: task = int(re_match.group(1)) @@ -46,23 +46,23 @@ def main(file, task_filter): method = re_match.group(3) size = int(re_match.group(4)) - if method == "try_grow" and "Err" in line: - # do not update allocation if try_grow failed - # annotate this entry so it can be shown in the chart - print(consumer, ",", alloc[consumer], ",ERR") - pass + if alloc.get(consumer) is None: + alloc[consumer] = size else: - if alloc.get(consumer) is None: - alloc[consumer] = size - else: - if method == "grow" or method == "try_grow": + if method == "grow" or method == "try_grow": + if "Err" in line: + # do not update allocation if try_grow failed + # annotate this entry so it can be shown in the chart + print(consumer, ",", alloc[consumer], ",ERR") + else: alloc[consumer] = alloc[consumer] + size - elif method == "shrink": - alloc[consumer] = alloc[consumer] - size - print(consumer, ",", alloc[consumer]) + elif method == "shrink": + alloc[consumer] = alloc[consumer] - size - except: - print("error parsing", line, file=sys.stderr) + print(consumer, ",", alloc[consumer]) + + except Exception as e: + print("error parsing", line, e, file=sys.stderr) if __name__ == "__main__": From 706f5e7aeb3e853985d2f0eb46ff3c7190498722 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 15:53:39 -0600 Subject: [PATCH 30/37] format --- dev/scripts/plot_memory_usage.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/scripts/plot_memory_usage.py b/dev/scripts/plot_memory_usage.py index 89c29d80fe..f6842970de 100644 --- a/dev/scripts/plot_memory_usage.py +++ b/dev/scripts/plot_memory_usage.py @@ -32,7 +32,7 @@ def plot_memory_usage(csv_file): plt.xlabel('Time') plt.ylabel('Memory Usage') plt.title('Memory Usage Over Time by Consumer') - plt.legend(loc='upper left') + plt.legend(loc='upper left', bbox_to_anchor=(1.05, 1), borderaxespad=0, fontsize='small') plt.grid(True, alpha=0.3) plt.tight_layout() From 4faf8818af67123ba67fd2eea864b66cea334004 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 16:13:46 -0600 Subject: [PATCH 31/37] ASF header --- dev/scripts/plot_memory_usage.py | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/dev/scripts/plot_memory_usage.py b/dev/scripts/plot_memory_usage.py index f6842970de..1ba82baeff 100644 --- a/dev/scripts/plot_memory_usage.py +++ b/dev/scripts/plot_memory_usage.py @@ -1,4 +1,23 @@ -#!/usr/bin/env python3 +#!/usr/bin/python +############################################################################## +# 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. +############################################################################## + import pandas as pd import matplotlib.pyplot as plt import sys From d91abdad8ec1f86d995be74469efcbb5c68cf30b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 17:18:41 -0600 Subject: [PATCH 32/37] add brief docs --- docs/source/contributor-guide/debugging.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/source/contributor-guide/debugging.md b/docs/source/contributor-guide/debugging.md index eaa73c9231..21c5a1eef6 100644 --- a/docs/source/contributor-guide/debugging.md +++ b/docs/source/contributor-guide/debugging.md @@ -189,3 +189,21 @@ This produces output like the following: Additionally, you can place a `log4rs.yaml` configuration file inside the Comet configuration directory specified by the `COMET_CONF_DIR` environment variable to enable more advanced logging configurations. This file uses the [log4rs YAML configuration format](https://docs.rs/log4rs/latest/log4rs/#configuration-via-a-yaml-file). For example, see: [log4rs.yaml](https://github.com/apache/datafusion-comet/blob/main/conf/log4rs.yaml). + +### Debugging Memory Reservations + +Set `spark.comet.debug.memory=true` to log all calls that grow or shrink memory reservations. + +There are Python scripts in `dev/scripts` that can be used to produce charts for a particular Spark task. + +First, extract the memory logging and write to CSV: + +```shell +python3 dev/scripts/mem_debug_to_csv.py /path/to/executor/log > /tmp/mem.csv +``` + +Next, generate a chart from the CSV file for a specific Spark task: + +```shell +python3 dev/scripts/plot_memory_usage.py /tmp/mem.csv --task 1234 +``` \ No newline at end of file From f6128b5d0e9e28c248d4daeb27b3a97072e784c2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 3 Oct 2025 17:27:30 -0600 Subject: [PATCH 33/37] docs --- .../scala/org/apache/comet/CometConf.scala | 6 ++++-- docs/source/contributor-guide/debugging.md | 20 +++++++++++++++++++ docs/source/user-guide/latest/configs.md | 1 + 3 files changed, 25 insertions(+), 2 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 3fa962d59b..e19d6834bd 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -54,6 +54,9 @@ object CometConf extends ShimCometConf { private val TRACING_GUIDE = "For more information, refer to the Comet Tracing " + "Guide (https://datafusion.apache.org/comet/user-guide/tracing.html)" + private val DEBUGGING_GUIDE = "For more information, refer to the Comet Debugging " + + "Guide (https://datafusion.apache.org/comet/contributor-guide/debugging.html" + /** List of all configs that is used for generating documentation */ val allConfs = new ListBuffer[ConfigEntry[_]] @@ -458,8 +461,7 @@ object CometConf extends ShimCometConf { val COMET_DEBUG_MEMORY_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_PREFIX.debug.memory") - .doc("When enabled, log all native memory pool interactions to stdout.") - .internal() + .doc(s"When enabled, log all native memory pool interactions. $DEBUGGING_GUIDE.") .booleanConf .createWithDefault(false) diff --git a/docs/source/contributor-guide/debugging.md b/docs/source/contributor-guide/debugging.md index 21c5a1eef6..00e1894f07 100644 --- a/docs/source/contributor-guide/debugging.md +++ b/docs/source/contributor-guide/debugging.md @@ -194,6 +194,26 @@ For example, see: [log4rs.yaml](https://github.com/apache/datafusion-comet/blob/ Set `spark.comet.debug.memory=true` to log all calls that grow or shrink memory reservations. +Example log output: + +``` +[Task 486] MemoryPool[ExternalSorter[6]].try_grow(256232960) returning Ok +[Task 486] MemoryPool[ExternalSorter[6]].try_grow(256375168) returning Ok +[Task 486] MemoryPool[ExternalSorter[6]].try_grow(256899456) returning Ok +[Task 486] MemoryPool[ExternalSorter[6]].try_grow(257296128) returning Ok +[Task 486] MemoryPool[ExternalSorter[6]].try_grow(257820416) returning Err +[Task 486] MemoryPool[ExternalSorterMerge[6]].shrink(10485760) +[Task 486] MemoryPool[ExternalSorter[6]].shrink(150464) +[Task 486] MemoryPool[ExternalSorter[6]].shrink(146688) +[Task 486] MemoryPool[ExternalSorter[6]].shrink(137856) +[Task 486] MemoryPool[ExternalSorter[6]].shrink(141952) +[Task 486] MemoryPool[ExternalSorterMerge[6]].try_grow(0) returning Ok +[Task 486] MemoryPool[ExternalSorterMerge[6]].try_grow(0) returning Ok +[Task 486] MemoryPool[ExternalSorter[6]].shrink(524288) +[Task 486] MemoryPool[ExternalSorterMerge[6]].try_grow(0) returning Ok +[Task 486] MemoryPool[ExternalSorterMerge[6]].try_grow(68928) returning Ok +``` + There are Python scripts in `dev/scripts` that can be used to produce charts for a particular Spark task. First, extract the memory logging and write to CSV: diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index bebca3c443..21fb143ed4 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -35,6 +35,7 @@ Comet provides the following configuration settings. | spark.comet.convert.json.enabled | When enabled, data from Spark (non-native) JSON v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | | spark.comet.convert.parquet.enabled | When enabled, data from Spark (non-native) Parquet v1 and v2 scans will be converted to Arrow format. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. | false | | spark.comet.debug.enabled | Whether to enable debug mode for Comet. When enabled, Comet will do additional checks for debugging purpose. For example, validating array when importing arrays from JVM at native side. Note that these checks may be expensive in performance and should only be enabled for debugging purpose. | false | +| spark.comet.debug.memory | When enabled, log all native memory pool interactions. For more information, refer to the Comet Debugging Guide (https://datafusion.apache.org/comet/contributor-guide/debugging.html. | false | | spark.comet.dppFallback.enabled | Whether to fall back to Spark for queries that use DPP. | true | | spark.comet.enabled | Whether to enable Comet extension for Spark. When this is turned on, Spark will use Comet to read Parquet data source. Note that to enable native vectorized execution, both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this config is the value of the env var `ENABLE_COMET` if set, or true otherwise. | true | | spark.comet.exceptionOnDatetimeRebase | Whether to throw exception when seeing dates/timestamps from the legacy hybrid (Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according to the Proleptic Gregorian calendar. When this is true, Comet will throw exceptions when seeing these dates/timestamps that were written by Spark version before 3.0. If this is false, these dates/timestamps will be read as if they were written to the Proleptic Gregorian calendar and will not be rebased. | false | From 7d40ac2a8af8858d3f95a9d414239779be4c3904 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 5 Oct 2025 13:00:46 -0600 Subject: [PATCH 34/37] fix --- dev/benchmarks/comet-tpcds.sh | 2 +- dev/benchmarks/comet-tpch.sh | 2 +- docs/source/contributor-guide/debugging.md | 4 +++- .../src/execution/memory_pools/logging_pool.rs | 14 +++++++++++++- 4 files changed, 18 insertions(+), 4 deletions(-) diff --git a/dev/benchmarks/comet-tpcds.sh b/dev/benchmarks/comet-tpcds.sh index cd44bb50a1..f8c17f5eef 100755 --- a/dev/benchmarks/comet-tpcds.sh +++ b/dev/benchmarks/comet-tpcds.sh @@ -22,7 +22,7 @@ $SPARK_HOME/sbin/stop-master.sh $SPARK_HOME/sbin/stop-worker.sh $SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER +RUST_BACKTRACE=1 $SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER $SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index e1f6f969ff..2b1623fa40 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -22,7 +22,7 @@ $SPARK_HOME/sbin/stop-master.sh $SPARK_HOME/sbin/stop-worker.sh $SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER +RUST_BACKTRACE=1 $SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER $SPARK_HOME/bin/spark-submit \ --master $SPARK_MASTER \ diff --git a/docs/source/contributor-guide/debugging.md b/docs/source/contributor-guide/debugging.md index 00e1894f07..0b8a25de56 100644 --- a/docs/source/contributor-guide/debugging.md +++ b/docs/source/contributor-guide/debugging.md @@ -128,7 +128,7 @@ To build Comet with this feature enabled: make release COMET_FEATURES=backtrace ``` -Start Comet with `RUST_BACKTRACE=1` +Set `RUST_BACKTRACE=1` for the Spark worker/executor process, or for `spark-submit` if running in local mode. ```console RUST_BACKTRACE=1 $SPARK_HOME/spark-shell --jars spark/target/comet-spark-spark3.5_2.12-$COMET_VERSION.jar --conf spark.plugins=org.apache.spark.CometPlugin --conf spark.comet.enabled=true --conf spark.comet.exec.enabled=true @@ -214,6 +214,8 @@ Example log output: [Task 486] MemoryPool[ExternalSorterMerge[6]].try_grow(68928) returning Ok ``` +When backtraces are enabled (see earlier section) then backtraces will be included for failed allocations. + There are Python scripts in `dev/scripts` that can be used to produce charts for a particular Spark task. First, extract the memory logging and write to CSV: diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index af37782b7d..4107ece273 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use datafusion::execution::memory_pool::{MemoryPool, MemoryReservation}; +use datafusion::execution::memory_pool::{MemoryConsumer, MemoryLimit, MemoryPool, MemoryReservation}; use log::info; use std::sync::Arc; @@ -35,6 +35,14 @@ impl LoggingPool { } impl MemoryPool for LoggingPool { + fn register(&self, consumer: &MemoryConsumer) { + self.pool.register(consumer) + } + + fn unregister(&self, consumer: &MemoryConsumer) { + self.pool.unregister(consumer) + } + fn grow(&self, reservation: &MemoryReservation, additional: usize) { info!( "[Task {}] MemoryPool[{}].grow({})", @@ -85,4 +93,8 @@ impl MemoryPool for LoggingPool { fn reserved(&self) -> usize { self.pool.reserved() } + + fn memory_limit(&self) -> MemoryLimit { + self.pool.memory_limit() + } } From c495897d0cfee5427237aaafafc28db033f6d5a4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 6 Oct 2025 08:35:57 -0600 Subject: [PATCH 35/37] cargo fmt --- native/core/src/execution/memory_pools/logging_pool.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index 4107ece273..8cd2ca6e55 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -15,7 +15,9 @@ // specific language governing permissions and limitations // under the License. -use datafusion::execution::memory_pool::{MemoryConsumer, MemoryLimit, MemoryPool, MemoryReservation}; +use datafusion::execution::memory_pool::{ + MemoryConsumer, MemoryLimit, MemoryPool, MemoryReservation, +}; use log::info; use std::sync::Arc; From e51751f29d66614f54ac0b704bf66518fdfa7292 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 6 Oct 2025 14:04:23 -0600 Subject: [PATCH 36/37] format --- native/core/src/execution/jni_api.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 91e4cad670..7b4c73cfa1 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -79,7 +79,10 @@ use crate::execution::spark_plan::SparkPlan; use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace}; use crate::execution::memory_pools::logging_pool::LoggingPool; -use crate::execution::spark_config::{SparkConfig, COMET_DEBUG_ENABLED, COMET_DEBUG_MEMORY, COMET_EXPLAIN_NATIVE_ENABLED, COMET_MAX_TEMP_DIRECTORY_SIZE, COMET_TRACING_ENABLED}; +use crate::execution::spark_config::{ + SparkConfig, COMET_DEBUG_ENABLED, COMET_DEBUG_MEMORY, COMET_EXPLAIN_NATIVE_ENABLED, + COMET_MAX_TEMP_DIRECTORY_SIZE, COMET_TRACING_ENABLED, +}; use datafusion_comet_proto::spark_operator::operator::OpStruct; use log::info; use once_cell::sync::Lazy; From e84428742cac37ad0fff2d7e293e75dbada70144 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 7 Oct 2025 14:15:43 -0600 Subject: [PATCH 37/37] fix regression --- native/core/src/execution/jni_api.rs | 3 ++- native/core/src/execution/spark_config.rs | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 2fbbe855d4..e868a5e784 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -186,7 +186,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED); let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED); let tracing_enabled = spark_config.get_bool(COMET_TRACING_ENABLED); - let max_temp_directory_size = spark_config.get_u64(COMET_MAX_TEMP_DIRECTORY_SIZE); + let max_temp_directory_size = + spark_config.get_u64(COMET_MAX_TEMP_DIRECTORY_SIZE, 100 * 1024 * 1024 * 1024); let logging_memory_pool = spark_config.get_bool(COMET_DEBUG_MEMORY); with_trace("createPlan", tracing_enabled, || { diff --git a/native/core/src/execution/spark_config.rs b/native/core/src/execution/spark_config.rs index 6aa9c1faf2..b257a5ba68 100644 --- a/native/core/src/execution/spark_config.rs +++ b/native/core/src/execution/spark_config.rs @@ -25,7 +25,7 @@ pub(crate) const COMET_DEBUG_MEMORY: &str = "spark.comet.debug.memory"; pub(crate) trait SparkConfig { fn get_bool(&self, name: &str) -> bool; - fn get_u64(&self, name: &str) -> u64; + fn get_u64(&self, name: &str, default_value: u64) -> u64; } impl SparkConfig for HashMap { @@ -35,9 +35,9 @@ impl SparkConfig for HashMap { .unwrap_or(false) } - fn get_u64(&self, name: &str) -> u64 { + fn get_u64(&self, name: &str, default_value: u64) -> u64 { self.get(name) .and_then(|str_val| str_val.parse::().ok()) - .unwrap_or(0) + .unwrap_or(default_value) } }