(incubator-gluten) branch main updated: [VL] Make ColumnarBatch::getRowBytes leak-safe (#6002)

2024-06-06 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new f72349ed8 [VL] Make ColumnarBatch::getRowBytes leak-safe (#6002)
f72349ed8 is described below

commit f72349ed8b18b40b45428a2c11bb658988c8e97c
Author: Hongze Zhang 
AuthorDate: Thu Jun 6 15:32:39 2024 +0800

[VL] Make ColumnarBatch::getRowBytes leak-safe (#6002)
---
 cpp/core/jni/JniWrapper.cc | 29 +++--
 cpp/core/memory/ColumnarBatch.cc   | 16 
 cpp/core/memory/ColumnarBatch.h|  9 +
 cpp/velox/memory/VeloxColumnarBatch.cc | 10 +-
 cpp/velox/memory/VeloxColumnarBatch.h  |  2 +-
 5 files changed, 30 insertions(+), 36 deletions(-)

diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc
index f5a6c4bd7..db498f43a 100644
--- a/cpp/core/jni/JniWrapper.cc
+++ b/cpp/core/jni/JniWrapper.cc
@@ -72,8 +72,8 @@ static jclass shuffleReaderMetricsClass;
 static jmethodID shuffleReaderMetricsSetDecompressTime;
 static jmethodID shuffleReaderMetricsSetDeserializeTime;
 
-static jclass block_stripes_class;
-static jmethodID block_stripes_constructor;
+static jclass blockStripesClass;
+static jmethodID blockStripesConstructor;
 
 class JavaInputStreamAdaptor final : public arrow::io::InputStream {
  public:
@@ -280,9 +280,9 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) {
   shuffleReaderMetricsSetDeserializeTime =
   getMethodIdOrError(env, shuffleReaderMetricsClass, "setDeserializeTime", 
"(J)V");
 
-  block_stripes_class =
+  blockStripesClass =
   createGlobalClassReferenceOrError(env, 
"Lorg/apache/spark/sql/execution/datasources/BlockStripes;");
-  block_stripes_constructor = env->GetMethodID(block_stripes_class, "", 
"(J[J[II[B)V");
+  blockStripesConstructor = env->GetMethodID(blockStripesClass, "", 
"(J[J[II[B)V");
 
   return jniVersion;
 }
@@ -297,7 +297,7 @@ void JNI_OnUnload(JavaVM* vm, void* reserved) {
   env->DeleteGlobalRef(nativeColumnarToRowInfoClass);
   env->DeleteGlobalRef(byteArrayClass);
   env->DeleteGlobalRef(shuffleReaderMetricsClass);
-  env->DeleteGlobalRef(block_stripes_class);
+  env->DeleteGlobalRef(blockStripesClass);
 
   gluten::getJniErrorState()->close();
   gluten::getJniCommonState()->close();
@@ -1224,14 +1224,13 @@ 
Java_org_apache_gluten_datasource_DatasourceJniWrapper_splitBlockByPartitionAndB
   }
 
   MemoryManager* memoryManager = 
reinterpret_cast(memoryManagerId);
-  auto result = batch->getRowBytes(0);
-  auto rowBytes = result.first;
+  auto result = batch->toUnsafeRow(0);
+  auto rowBytes = result.data();
   auto newBatchHandle = ctx->objectStore()->save(ctx->select(memoryManager, 
batch, partitionColIndiceVec));
 
-  auto bytesSize = result.second;
+  auto bytesSize = result.size();
   jbyteArray bytesArray = env->NewByteArray(bytesSize);
   env->SetByteArrayRegion(bytesArray, 0, bytesSize, 
reinterpret_cast(rowBytes));
-  delete[] rowBytes;
 
   jlongArray batchArray = env->NewLongArray(1);
   long* cBatchArray = new long[1];
@@ -1239,15 +1238,9 @@ 
Java_org_apache_gluten_datasource_DatasourceJniWrapper_splitBlockByPartitionAndB
   env->SetLongArrayRegion(batchArray, 0, 1, cBatchArray);
   delete[] cBatchArray;
 
-  jobject block_stripes = env->NewObject(
-  block_stripes_class,
-  block_stripes_constructor,
-  batchHandle,
-  batchArray,
-  nullptr,
-  batch->numColumns(),
-  bytesArray);
-  return block_stripes;
+  jobject blockStripes = env->NewObject(
+  blockStripesClass, blockStripesConstructor, batchHandle, batchArray, 
nullptr, batch->numColumns(), bytesArray);
+  return blockStripes;
   JNI_METHOD_END(nullptr)
 }
 
diff --git a/cpp/core/memory/ColumnarBatch.cc b/cpp/core/memory/ColumnarBatch.cc
index bb80510ee..23567535d 100644
--- a/cpp/core/memory/ColumnarBatch.cc
+++ b/cpp/core/memory/ColumnarBatch.cc
@@ -43,8 +43,8 @@ int64_t ColumnarBatch::getExportNanos() const {
   return exportNanos_;
 }
 
-std::pair ColumnarBatch::getRowBytes(int32_t rowId) const {
-  throw gluten::GlutenException("Not implemented getRowBytes for 
ColumnarBatch");
+std::vector ColumnarBatch::toUnsafeRow(int32_t rowId) const {
+  throw gluten::GlutenException("Not implemented toUnsafeRow for 
ColumnarBatch");
 }
 
 std::ostream& operator<<(std::ostream& os, const ColumnarBatch& columnarBatch) 
{
@@ -86,8 +86,8 @@ std::shared_ptr 
ArrowColumnarBatch::exportArrowArray() {
   return cArray;
 }
 
-std::pair ArrowColumnarBatch::getRowBytes(int32_t rowId) const {
-  throw gluten::GlutenException("Not implemented getRowBytes for 
ArrowColumnarBatch");
+std::vector ArrowColumnarBatch::toUnsafeRow(int32_t rowId) const {
+  

(incubator-gluten) branch main updated: [VL] Do not skip updating children's metrics while visiting an operator with NoopMetricsUpdater (#5933)

2024-06-05 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 15c7c5808 [VL] Do not skip updating children's metrics while visiting 
an operator with NoopMetricsUpdater (#5933)
15c7c5808 is described below

commit 15c7c5808eb26468b9fe0e237d5e5edf26490fa6
Author: Hongze Zhang 
AuthorDate: Wed Jun 5 16:20:58 2024 +0800

[VL] Do not skip updating children's metrics while visiting an operator 
with NoopMetricsUpdater (#5933)
---
 .../metrics/HashAggregateMetricsUpdater.scala  |  2 +-
 .../gluten/metrics/HashJoinMetricsUpdater.scala|  2 +-
 .../org/apache/gluten/metrics/MetricsUtil.scala|  6 ++--
 .../apache/gluten/execution/TopNTransformer.scala  |  4 +--
 .../gluten/execution/VeloxMetricsSuite.scala   | 36 ++
 .../gluten/execution/WholeStageTransformer.scala   |  6 ++--
 .../columnar/enumerated/RemoveFilter.scala |  6 ++--
 .../org/apache/gluten/metrics/MetricsUpdater.scala | 31 ++-
 .../org/apache/gluten/metrics/MetricsUtil.scala| 36 --
 9 files changed, 93 insertions(+), 36 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashAggregateMetricsUpdater.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashAggregateMetricsUpdater.scala
index e2014e5b8..b035d7a04 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashAggregateMetricsUpdater.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashAggregateMetricsUpdater.scala
@@ -65,7 +65,7 @@ class HashAggregateMetricsUpdater(val metrics: Map[String, 
SQLMetric])
 }
   }
 } catch {
-  case e: Throwable =>
+  case e: Exception =>
 logError(s"Updating native metrics failed due to ${e.getCause}.")
 throw e
 }
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashJoinMetricsUpdater.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashJoinMetricsUpdater.scala
index 3c35286c1..ca891bac2 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashJoinMetricsUpdater.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/HashJoinMetricsUpdater.scala
@@ -104,7 +104,7 @@ class HashJoinMetricsUpdater(val metrics: Map[String, 
SQLMetric])
 }
   }
 } catch {
-  case e: Throwable =>
+  case e: Exception =>
 logError(s"Updating native metrics failed due to ${e.getCause}.")
 throw e
 }
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
index a6dfb3dbc..1376dc6a8 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
@@ -41,7 +41,7 @@ object MetricsUtil extends Logging {
   case t: TransformSupport =>
 MetricsUpdaterTree(t.metricsUpdater(), 
t.children.map(treeifyMetricsUpdaters))
   case _ =>
-MetricsUpdaterTree(NoopMetricsUpdater, Seq())
+MetricsUpdaterTree(MetricsUpdater.Terminate, Seq())
 }
   }
 
@@ -107,7 +107,7 @@ object MetricsUtil extends Logging {
 s"Updating native metrics failed due to the wrong size of metrics 
data: " +
   s"$numNativeMetrics")
   ()
-} else if (mutNode.updater == NoopMetricsUpdater) {
+} else if (mutNode.updater == MetricsUpdater.Terminate) {
   ()
 } else {
   updateTransformerMetricsInternal(
@@ -159,7 +159,7 @@ object MetricsUtil extends Logging {
 
 mutNode.children.foreach {
   child =>
-if (child.updater != NoopMetricsUpdater) {
+if (child.updater != MetricsUpdater.Terminate) {
   val result = updateTransformerMetricsInternal(
 child,
 relMap,
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala
index c2d12415c..01c89bee2 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala
@@ -19,7 +19,7 @@ package org.apache.gluten.execution
 import org.apache.gluten.backendsapi.BackendsApiManager
 import org.apache.gluten.expression.{ConverterUtils, ExpressionConverter}
 import org.apache.gluten.extension.ValidationResult
-import org.apache.gluten.metrics.{MetricsUpdater, NoopMetricsUpdater}
+import org.apache.gluten.metrics.MetricsUpdater
 import org.apache.gluten.substrait.`type`.Typ

(incubator-gluten) branch main updated: [VL] Quick fix for Uniffle CI error (#5986)

2024-06-04 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 59aaa1cc3 [VL] Quick fix for Uniffle CI error (#5986)
59aaa1cc3 is described below

commit 59aaa1cc36686b34900b44b920679ed335f94302
Author: Hongze Zhang 
AuthorDate: Wed Jun 5 11:41:36 2024 +0800

[VL] Quick fix for Uniffle CI error (#5986)
---
 .../org/apache/gluten/integration/BaseMixin.java   |  2 +-
 .../org/apache/gluten/integration/Suite.scala  | 22 --
 2 files changed, 13 insertions(+), 11 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
index dc1691e50..93c82a6fa 100644
--- 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
+++ 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
@@ -75,7 +75,7 @@ public class BaseMixin {
   @CommandLine.Option(names = {"--shuffle-partitions"}, description = "Shuffle 
partition number", defaultValue = "100")
   private int shufflePartitions;
 
-  @CommandLine.Option(names = {"--scan-partitions"}, description = "Scan 
partition number. This is an approximate value, so the actual scan partition 
number might vary around this value", defaultValue = "100")
+  @CommandLine.Option(names = {"--scan-partitions"}, description = "Scan 
partition number. This is an approximate value, so the actual scan partition 
number might vary around this value. -1 for letting Spark choose an appropriate 
number.", defaultValue = "-1")
   private int scanPartitions;
 
   @CommandLine.Option(names = {"--extra-conf"}, description = "Extra Spark 
config entries applying to generated Spark session. E.g. --extra-conf=k1=v1 
--extra-conf=k2=v2")
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
index bb5cb1889..51e1777e2 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
@@ -103,16 +103,18 @@ abstract class Suite(
 
sessionSwitcher.defaultConf().setWarningOnOverriding("spark.sql.codegen.wholeStage",
 "false")
   }
 
-  // Scan partition number.
-  sessionSwitcher
-.defaultConf()
-.setWarningOnOverriding("spark.sql.files.maxPartitionBytes", 
s"${ByteUnit.PiB.toBytes(1L)}")
-  sessionSwitcher
-.defaultConf()
-.setWarningOnOverriding("spark.sql.files.openCostInBytes", "0")
-  sessionSwitcher
-.defaultConf()
-.setWarningOnOverriding("spark.sql.files.minPartitionNum", 
s"${(scanPartitions - 1) max 1}")
+  if (scanPartitions != -1) {
+// Scan partition number.
+sessionSwitcher
+  .defaultConf()
+  .setWarningOnOverriding("spark.sql.files.maxPartitionBytes", 
s"${ByteUnit.PiB.toBytes(1L)}")
+sessionSwitcher
+  .defaultConf()
+  .setWarningOnOverriding("spark.sql.files.openCostInBytes", "0")
+sessionSwitcher
+  .defaultConf()
+  .setWarningOnOverriding("spark.sql.files.minPartitionNum", 
s"${(scanPartitions - 1) max 1}")
+  }
 
   extraSparkConf.toStream.foreach { kv =>
 sessionSwitcher.defaultConf().setWarningOnOverriding(kv._1, kv._2)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Gluten-it: Add option --scan-partitions (#5958)

2024-06-03 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 785958e0d [VL] Gluten-it: Add option --scan-partitions (#5958)
785958e0d is described below

commit 785958e0d3693cf7529237119a6c918d8639a833
Author: Hongze Zhang 
AuthorDate: Tue Jun 4 10:42:49 2024 +0800

[VL] Gluten-it: Add option --scan-partitions (#5958)
---
 .../org/apache/gluten/integration/BaseMixin.java| 12 ++--
 .../scala/org/apache/gluten/integration/Suite.scala | 21 +++--
 .../integration/clickbench/ClickBenchSuite.scala|  4 ++--
 .../apache/gluten/integration/ds/TpcdsSuite.scala   |  4 ++--
 .../org/apache/gluten/integration/h/TpchSuite.scala |  4 ++--
 5 files changed, 23 insertions(+), 22 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
index 41d244871..dc1691e50 100644
--- 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
+++ 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java
@@ -72,11 +72,11 @@ public class BaseMixin {
   @CommandLine.Option(names = {"--disable-wscg"}, description = "Disable Spark 
SQL whole stage code generation", defaultValue = "false")
   private boolean disableWscg;
 
-  @CommandLine.Option(names = {"--shuffle-partitions"}, description = 
"Generate data with partitions", defaultValue = "100")
+  @CommandLine.Option(names = {"--shuffle-partitions"}, description = "Shuffle 
partition number", defaultValue = "100")
   private int shufflePartitions;
 
-  @CommandLine.Option(names = {"--min-scan-partitions"}, description = "Use 
minimum number of partitions to read data", defaultValue = "false")
-  private boolean minimumScanPartitions;
+  @CommandLine.Option(names = {"--scan-partitions"}, description = "Scan 
partition number. This is an approximate value, so the actual scan partition 
number might vary around this value", defaultValue = "100")
+  private int scanPartitions;
 
   @CommandLine.Option(names = {"--extra-conf"}, description = "Extra Spark 
config entries applying to generated Spark session. E.g. --extra-conf=k1=v1 
--extra-conf=k2=v2")
   private Map extraSparkConf = Collections.emptyMap();
@@ -131,19 +131,19 @@ public class BaseMixin {
 suite = new TpchSuite(runModeEnumeration.getSparkMasterUrl(), actions, 
testConf,
 baselineConf, extraSparkConfScala, level, errorOnMemLeak, enableUi,
 enableHsUi, hsUiPort, offHeapSize, disableAqe, disableBhj,
-disableWscg, shufflePartitions, minimumScanPartitions);
+disableWscg, shufflePartitions, scanPartitions);
 break;
   case "ds":
 suite = new TpcdsSuite(runModeEnumeration.getSparkMasterUrl(), 
actions, testConf,
 baselineConf, extraSparkConfScala, level, errorOnMemLeak, enableUi,
 enableHsUi, hsUiPort, offHeapSize, disableAqe, disableBhj,
-disableWscg, shufflePartitions, minimumScanPartitions);
+disableWscg, shufflePartitions, scanPartitions);
 break;
   case "clickbench":
 suite = new ClickBenchSuite(runModeEnumeration.getSparkMasterUrl(), 
actions, testConf,
 baselineConf, extraSparkConfScala, level, errorOnMemLeak, enableUi,
 enableHsUi, hsUiPort, offHeapSize, disableAqe, disableBhj,
-disableWscg, shufflePartitions, minimumScanPartitions);
+disableWscg, shufflePartitions, scanPartitions);
 break;
   default:
 throw new IllegalArgumentException("TPC benchmark type not found: " + 
benchmarkType);
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
index 9e31e1171..bb5cb1889 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala
@@ -43,7 +43,7 @@ abstract class Suite(
 private val disableBhj: Boolean,
 private val disableWscg: Boolean,
 private val shufflePartitions: Int,
-private val minimumScanPartitions: Boolean) {
+private val scanPartitions: Int) {
 
   resetLogLevel()
 
@@ -103,15 +103,16 @@ abstract class Suite(
 
sessionSwitcher.defaultConf().setWarningOnOverriding("spark.sql.codegen.wholeStage",
 "false")
   }
 
-  if (minimumScanPartitions) {
-sessionSwitcher
-  .defaultConf()
-  .setWarning

(incubator-gluten) branch main updated (7a036dd6e -> ad817ed51)

2024-06-02 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from 7a036dd6e [GLUTEN-5939][CH] Support java timezone id named 'GMT+8' or 
'GMT+08:00' (#5940)
 add ad817ed51 [GLUTEN-5414] [VL] Support arrow csv option and schema 
(#5850)

No new revisions were added by this update.

Summary of changes:
 .github/workflows/velox_docker.yml |  71 +-
 .github/workflows/velox_docker_cache.yml   |   8 +-
 .../gluten/datasource/ArrowCSVFileFormat.scala | 111 ++-
 .../datasource/ArrowCSVOptionConverter.scala   |  62 ++
 .../gluten/datasource/ArrowConvertorRule.scala |  12 +-
 .../v2/ArrowCSVPartitionReaderFactory.scala|  79 +-
 .../gluten/datasource/v2/ArrowCSVTable.scala   |  12 +
 .../gluten/extension/ArrowScanReplaceRule.scala|   3 +-
 .../resources/datasource/csv/student_option.csv|   4 +
 .../datasource/csv/student_option_schema.csv   |   4 +
 .../csv/{student.csv => student_option_str.csv}|   2 +-
 .../org/apache/gluten/execution/TestOperator.scala | 235 --
 ep/build-velox/src/build_velox.sh  |  10 +-
 ep/build-velox/src/get_velox.sh|   2 +
 .../src/modify_arrow_dataset_scan_option.patch | 883 +
 ep/build-velox/src/modify_velox.patch  |   3 +-
 .../execution/WholeStageTransformerSuite.scala |  20 +
 gluten-data/pom.xml|   4 +-
 .../scala/org/apache/gluten/utils/ArrowUtil.scala  | 147 +---
 gluten-ut/spark32/pom.xml  |   2 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |   9 +
 gluten-ut/spark33/pom.xml  |   2 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |   9 +
 gluten-ut/spark34/pom.xml  |   2 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |   9 +
 gluten-ut/spark35/pom.xml  |   2 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |  11 +
 .../execution/datasources/csv/GlutenCSVSuite.scala |  65 +-
 gluten-ut/test/pom.xml |   2 +-
 pom.xml|   1 +
 30 files changed, 1472 insertions(+), 314 deletions(-)
 create mode 100644 
backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowCSVOptionConverter.scala
 create mode 100644 
backends-velox/src/test/resources/datasource/csv/student_option.csv
 create mode 100644 
backends-velox/src/test/resources/datasource/csv/student_option_schema.csv
 copy backends-velox/src/test/resources/datasource/csv/{student.csv => 
student_option_str.csv} (78%)
 create mode 100644 ep/build-velox/src/modify_arrow_dataset_scan_option.patch


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Gluten-it: Optimize Maven dependency list

2024-05-30 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 4c9131a51 [VL] Gluten-it: Optimize Maven dependency list
4c9131a51 is described below

commit 4c9131a51f11e0f6f4796d58ca75d0ee08a01c09
Author: Hongze Zhang 
AuthorDate: Fri May 31 09:29:04 2024 +0800

[VL] Gluten-it: Optimize Maven dependency list
---
 tools/gluten-it/common/pom.xml  | 11 +
 tools/gluten-it/package/pom.xml | 11 +
 tools/gluten-it/pom.xml | 95 +
 3 files changed, 41 insertions(+), 76 deletions(-)

diff --git a/tools/gluten-it/common/pom.xml b/tools/gluten-it/common/pom.xml
index 4cd468472..de0d7c2e6 100644
--- a/tools/gluten-it/common/pom.xml
+++ b/tools/gluten-it/common/pom.xml
@@ -57,6 +57,17 @@
   provided
   test-jar
 
+
+  org.apache.spark
+  spark-sql_${scala.binary.version}
+  provided
+
+
+  org.apache.spark
+  spark-sql_${scala.binary.version}
+  provided
+  test-jar
+
 
 
   io.trino.tpcds
diff --git a/tools/gluten-it/package/pom.xml b/tools/gluten-it/package/pom.xml
index 1f86ee723..70a59cac0 100644
--- a/tools/gluten-it/package/pom.xml
+++ b/tools/gluten-it/package/pom.xml
@@ -69,6 +69,17 @@
   runtime
   test-jar
 
+
+  org.apache.spark
+  spark-sql_${scala.binary.version}
+  runtime
+
+
+  org.apache.spark
+  spark-sql_${scala.binary.version}
+  runtime
+  test-jar
+
   
 
   
diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml
index c0e2fc321..ccb59ade8 100644
--- a/tools/gluten-it/pom.xml
+++ b/tools/gluten-it/pom.xml
@@ -87,6 +87,25 @@
 provided
 test-jar
   
+  
+org.apache.spark
+spark-sql_${scala.binary.version}
+${spark.version}
+
+  
+com.google.protobuf
+protobuf-java
+  
+
+provided
+  
+  
+org.apache.spark
+spark-sql_${scala.binary.version}
+${spark.version}
+test-jar
+provided
+  
 
   
 
@@ -117,100 +136,24 @@
   
 3.2.2
   
-  
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  
-
-  com.google.protobuf
-  protobuf-java
-
-  
-
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  test-jar
-
-  
 
 
   spark-3.3
   
 3.3.1
   
-  
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  
-
-  com.google.protobuf
-  protobuf-java
-
-  
-
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  test-jar
-
-  
 
 
   spark-3.4
   
 3.4.2
   
-  
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  
-
-  com.google.protobuf
-  protobuf-java
-
-  
-
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  test-jar
-
-  
 
 
   spark-3.5
   
 3.5.1
   
-  
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  
-
-  com.google.protobuf
-  protobuf-java
-
-  
-
-
-  org.apache.spark
-  spark-sql_${scala.binary.version}
-  ${spark.version}
-  test-jar
-
-  
 
 
   celeborn-0.4


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Following #5889, correct / simplify the table indenting algorithm (#5917)

2024-05-29 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 1fd19a563 [VL] Following #5889, correct / simplify the table indenting 
algorithm (#5917)
1fd19a563 is described below

commit 1fd19a563b7ce1e2b0a0cb5a79a4044e743d3ab0
Author: Hongze Zhang 
AuthorDate: Thu May 30 11:00:06 2024 +0800

[VL] Following #5889, correct / simplify the table indenting algorithm 
(#5917)
---
 .../gluten/integration/action/Parameterized.scala  |  2 +-
 .../apache/gluten/integration/action/TableRender.scala | 10 ++
 .../gluten/integration/action/TableRenderTest.scala| 18 ++
 3 files changed, 25 insertions(+), 5 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
index 799b7632e..8f5bc0946 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
@@ -227,7 +227,7 @@ case class TestResultLines(
 metricNames: Seq[String],
 lines: Iterable[TestResultLine]) {
   def print(): Unit = {
-val fields = ListBuffer[String]("Query ID", "Succeed")
+val fields = ListBuffer[String]("Query ID", "Succeeded")
 dimNames.foreach(dimName => fields.append(dimName))
 metricNames.foreach(metricName => fields.append(metricName))
 fields.append("Row Count")
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala
index b25a5db93..4cded2848 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala
@@ -116,12 +116,14 @@ object TableRender {
 def updateWidth(field: Field, lowerBound: Int): Unit = {
   field match {
 case branch @ Field.Branch(name, children) =>
-  val childLowerBound =
-Math.ceil((lowerBound max name.length + 2).toDouble / 
children.size.toDouble).toInt
-  children.foreach(child => updateWidth(child, childLowerBound))
+  val leafLowerBound =
+Math
+  .ceil((lowerBound max name.length + 2).toDouble / 
branch.leafs.size.toDouble)
+  .toInt
+  children.foreach(child => updateWidth(child, leafLowerBound * 
child.leafs.size))
   val childrenWidth =
 children.map(child => 
widthMap(System.identityHashCode(child))).sum
-  val width = childLowerBound * children.size max childrenWidth + 
children.size - 1
+  val width = childrenWidth + children.size - 1
   val hash = System.identityHashCode(branch)
   widthMap += hash -> width
 case leaf @ Field.Leaf(name) =>
diff --git 
a/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala
 
b/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala
index 87ad23f36..ce7b0974c 100644
--- 
a/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala
+++ 
b/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala
@@ -82,10 +82,28 @@ object TableRenderTest {
 Console.out.println()
   }
 
+  def case4(): Unit = {
+val render: TableRender[Seq[String]] = TableRender.create(
+  Branch(
+"AC",
+List(Branch("AB", List(Leaf("A"), Leaf("B"))), Leaf("C"))),
+  Branch("DE", List(Leaf("D"), Leaf("E"(new RowParser[Seq[String]] {
+  override def parse(rowFactory: FieldAppender.RowAppender, row: 
Seq[String]): Unit = {
+val inc = rowFactory.incremental()
+row.foreach(ceil => inc.next().write(ceil))
+  }
+})
+
+render.appendRow(List("", "b", "cc", "d", "e"))
+render.print(Console.out)
+Console.out.println()
+  }
+
   def main(args: Array[String]): Unit = {
 case0()
 case1()
 case2()
 case3()
+case4()
   }
 }


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Gluten-it: Improve test report table rendering (#5889)

2024-05-29 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new efb81e284 [VL] Gluten-it: Improve test report table rendering (#5889)
efb81e284 is described below

commit efb81e2847da13cf51f5df11cea41b083b8f0475
Author: Hongze Zhang 
AuthorDate: Wed May 29 16:39:35 2024 +0800

[VL] Gluten-it: Improve test report table rendering (#5889)
---
 .../gluten/integration/command/SparkRunModes.java  |   3 +
 .../gluten/integration/action/Parameterized.scala  |  28 +-
 .../apache/gluten/integration/action/Queries.scala |  23 +-
 .../gluten/integration/action/QueriesCompare.scala |  50 ++--
 .../gluten/integration/action/TableFormatter.scala |  78 --
 .../gluten/integration/action/TableRender.scala| 308 +
 .../integration/action/TableRenderTest.scala   |  91 ++
 7 files changed, 454 insertions(+), 127 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java
 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java
index f5a5c73a6..cfd3848d8 100644
--- 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java
+++ 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java
@@ -140,6 +140,9 @@ public final class SparkRunModes {
   Optional extraClassPath = 
Arrays.stream(classPathValues).filter(classPath -> {
 File file = new File(classPath);
 return file.exists() && file.isFile() && 
extraJarSet.contains(file.getName());
+  }).map(classPath -> {
+File file = new File(classPath);
+return file.getAbsolutePath();
   }).reduce((s1, s2) -> s1 + File.pathSeparator + s2);
 
   final Map extras = new HashMap<>();
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
index 2871ef2de..799b7632e 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala
@@ -18,13 +18,14 @@ package org.apache.gluten.integration.action
 
 import org.apache.commons.lang3.exception.ExceptionUtils
 import org.apache.gluten.integration.action.Actions.QuerySelector
+import 
org.apache.gluten.integration.action.TableRender.RowParser.FieldAppender.RowAppender
 import org.apache.gluten.integration.stat.RamStat
 import org.apache.gluten.integration.{QueryRunner, Suite, TableCreator}
 import org.apache.spark.sql.ConfUtils.ConfImplicits._
 import org.apache.spark.sql.SparkSessionSwitcher
 
 import scala.collection.mutable
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.mutable.ListBuffer
 
 class Parameterized(
 scale: Double,
@@ -198,24 +199,25 @@ case class TestResultLine(
 
 object TestResultLine {
   class Parser(dimNames: Seq[String], metricNames: Seq[String])
-  extends TableFormatter.RowParser[TestResultLine] {
-override def parse(line: TestResultLine): Seq[Any] = {
-  val values = ArrayBuffer[Any](line.queryId, line.succeed)
+  extends TableRender.RowParser[TestResultLine] {
+override def parse(rowAppender: RowAppender, line: TestResultLine): Unit = 
{
+  val inc = rowAppender.incremental()
+  inc.next().write(line.queryId)
+  inc.next().write(line.succeed)
   dimNames.foreach { dimName =>
 val coordinate = line.coordinate.coordinate
 if (!coordinate.contains(dimName)) {
   throw new IllegalStateException("Dimension name not found" + dimName)
 }
-values.append(coordinate(dimName))
+inc.next().write(coordinate(dimName))
   }
   metricNames.foreach { metricName =>
 val metrics = line.metrics
-values.append(metrics.getOrElse(metricName, "N/A"))
+inc.next().write(metrics.getOrElse(metricName, "N/A"))
   }
-  values.append(line.rowCount.getOrElse("N/A"))
-  values.append(line.planningTimeMillis.getOrElse("N/A"))
-  values.append(line.executionTimeMillis.getOrElse("N/A"))
-  values
+  inc.next().write(line.rowCount.getOrElse("N/A"))
+  inc.next().write(line.planningTimeMillis.getOrElse("N/A"))
+  inc.next().write(line.executionTimeMillis.getOrElse("N/A"))
 }
   }
 }
@@ -231,14 +233,14 @@ case class TestResultLines(
 fields.append("Row Count")
 fields.append("Planning Time (Millis)")
 fields.append("Query Time (Millis)")
-val formatter = 

(incubator-gluten) branch main updated: [GLUTEN-4422][CORE] Fix core dump caused by spill on closed iterator (#5874)

2024-05-28 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 79c681b28 [GLUTEN-4422][CORE] Fix core dump caused by spill on closed 
iterator (#5874)
79c681b28 is described below

commit 79c681b2847fccd43292d3d735bc3972eacb3b52
Author: WangGuangxin 
AuthorDate: Wed May 29 10:26:32 2024 +0800

[GLUTEN-4422][CORE] Fix core dump caused by spill on closed iterator (#5874)

Closes #4422
---
 cpp/core/jni/JniWrapper.cc  | 4 
 .../java/org/apache/gluten/vectorized/ColumnarBatchOutIterator.java | 6 +-
 2 files changed, 9 insertions(+), 1 deletion(-)

diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc
index a04ba73a1..f5a6c4bd7 100644
--- a/cpp/core/jni/JniWrapper.cc
+++ b/cpp/core/jni/JniWrapper.cc
@@ -544,6 +544,10 @@ JNIEXPORT jlong JNICALL 
Java_org_apache_gluten_vectorized_ColumnarBatchOutIterat
   auto ctx = gluten::getRuntime(env, wrapper);
 
   auto it = ctx->objectStore()->retrieve(iterHandle);
+  if (it == nullptr) {
+std::string errorMessage = "Invalid result iter handle " + 
std::to_string(iterHandle);
+throw gluten::GlutenException(errorMessage);
+  }
   return it->spillFixedSize(size);
   JNI_METHOD_END(kInvalidResourceHandle)
 }
diff --git 
a/gluten-data/src/main/java/org/apache/gluten/vectorized/ColumnarBatchOutIterator.java
 
b/gluten-data/src/main/java/org/apache/gluten/vectorized/ColumnarBatchOutIterator.java
index 82b398439..37de98943 100644
--- 
a/gluten-data/src/main/java/org/apache/gluten/vectorized/ColumnarBatchOutIterator.java
+++ 
b/gluten-data/src/main/java/org/apache/gluten/vectorized/ColumnarBatchOutIterator.java
@@ -80,7 +80,11 @@ public class ColumnarBatchOutIterator extends 
GeneralOutIterator implements Runt
   }
 
   public long spill(long size) {
-return nativeSpill(iterHandle, size);
+if (!closed.get()) {
+  return nativeSpill(iterHandle, size);
+} else {
+  return 0L;
+}
   }
 
   @Override


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Include ClickBench benchmark in gluten-it (#5887)

2024-05-28 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 8f0440516 [VL] Include ClickBench benchmark in gluten-it (#5887)
8f0440516 is described below

commit 8f044051674e2ecb2fa378b3f01e9bf121cfad4f
Author: Hongze Zhang 
AuthorDate: Tue May 28 15:01:46 2024 +0800

[VL] Include ClickBench benchmark in gluten-it (#5887)
---
 tools/gluten-it/README.md  |   2 +-
 .../{tpc/TpcMixin.java => BaseMixin.java}  |  23 ++--
 .../gluten/integration/{tpc/Tpc.java => Cli.java}  |  20 +--
 .../{tpc => }/command/DataGenMixin.java|  10 +-
 .../integration/{tpc => }/command/DataGenOnly.java |   6 +-
 .../{tpc => }/command/Parameterized.java   |  18 +--
 .../integration/{tpc => }/command/Queries.java |  10 +-
 .../{tpc => }/command/QueriesCompare.java  |  11 +-
 .../{tpc => }/command/QueriesMixin.java|  10 +-
 .../{tpc => }/command/SparkRunModes.java   |   2 +-
 .../integration/{tpc => }/command/SparkShell.java  |  11 +-
 .../src/main/resources/clickbench-queries/q1.sql   |   1 +
 .../src/main/resources/clickbench-queries/q10.sql  |   1 +
 .../src/main/resources/clickbench-queries/q11.sql  |   1 +
 .../src/main/resources/clickbench-queries/q12.sql  |   1 +
 .../src/main/resources/clickbench-queries/q13.sql  |   1 +
 .../src/main/resources/clickbench-queries/q14.sql  |   1 +
 .../src/main/resources/clickbench-queries/q15.sql  |   1 +
 .../src/main/resources/clickbench-queries/q16.sql  |   1 +
 .../src/main/resources/clickbench-queries/q17.sql  |   1 +
 .../src/main/resources/clickbench-queries/q18.sql  |   1 +
 .../src/main/resources/clickbench-queries/q19.sql  |   1 +
 .../src/main/resources/clickbench-queries/q2.sql   |   1 +
 .../src/main/resources/clickbench-queries/q20.sql  |   1 +
 .../src/main/resources/clickbench-queries/q21.sql  |   1 +
 .../src/main/resources/clickbench-queries/q22.sql  |   1 +
 .../src/main/resources/clickbench-queries/q23.sql  |   1 +
 .../src/main/resources/clickbench-queries/q24.sql  |   1 +
 .../src/main/resources/clickbench-queries/q25.sql  |   1 +
 .../src/main/resources/clickbench-queries/q26.sql  |   1 +
 .../src/main/resources/clickbench-queries/q27.sql  |   1 +
 .../src/main/resources/clickbench-queries/q28.sql  |   1 +
 .../src/main/resources/clickbench-queries/q29.sql  |   1 +
 .../src/main/resources/clickbench-queries/q3.sql   |   1 +
 .../src/main/resources/clickbench-queries/q30.sql  |   1 +
 .../src/main/resources/clickbench-queries/q31.sql  |   1 +
 .../src/main/resources/clickbench-queries/q32.sql  |   1 +
 .../src/main/resources/clickbench-queries/q33.sql  |   1 +
 .../src/main/resources/clickbench-queries/q34.sql  |   1 +
 .../src/main/resources/clickbench-queries/q35.sql  |   1 +
 .../src/main/resources/clickbench-queries/q36.sql  |   1 +
 .../src/main/resources/clickbench-queries/q37.sql  |   1 +
 .../src/main/resources/clickbench-queries/q38.sql  |   1 +
 .../src/main/resources/clickbench-queries/q39.sql  |   1 +
 .../src/main/resources/clickbench-queries/q4.sql   |   1 +
 .../src/main/resources/clickbench-queries/q40.sql  |   1 +
 .../src/main/resources/clickbench-queries/q41.sql  |   1 +
 .../src/main/resources/clickbench-queries/q42.sql  |   1 +
 .../src/main/resources/clickbench-queries/q43.sql  |   1 +
 .../src/main/resources/clickbench-queries/q5.sql   |   1 +
 .../src/main/resources/clickbench-queries/q6.sql   |   1 +
 .../src/main/resources/clickbench-queries/q7.sql   |   1 +
 .../src/main/resources/clickbench-queries/q8.sql   |   1 +
 .../src/main/resources/clickbench-queries/q9.sql   |   1 +
 .../gluten/integration/{tpc => }/Constants.scala   |   2 +-
 .../gluten/integration/{tpc => }/DataGen.scala |   2 +-
 .../{tpc/TpcRunner.scala => QueryRunner.scala} |  39 ++
 .../gluten/integration/{tpc => }/ShimUtils.scala   |   2 +-
 .../{tpc/TpcSuite.scala => Suite.scala}|  32 +++--
 .../apache/gluten/integration/TableCreator.scala   |  50 +++
 .../integration/{tpc => }/action/Actions.scala |   8 +-
 .../integration/{tpc => }/action/DataGenOnly.scala |  10 +-
 .../{tpc => }/action/Parameterized.scala   |  48 ---
 .../integration/{tpc => }/action/Queries.scala |  32 +++--
 .../{tpc => }/action/QueriesCompare.scala  |  36 ++---
 .../integration/{tpc => }/action/SparkShell.scala  |  20 +--
 .../{tpc => }/action/TableFormatter.scala  |   2 +-
 .../integration/clickbench/ClickBenchDataGen.scala |  45 +++
 .../ClickBenchSuite.scala} |  88 +---
 .../clickbench/ClickBenchTableCreator.scala| 150 +
 .../integration/{tpc => }/ds/TpcdsDataGen.scala|   4 +-
 .../integration/{tpc => }/ds/TpcdsSui

(incubator-gluten) branch main updated: [VL] Following #5861, append some nit changes

2024-05-27 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 12fdb7029 [VL] Following #5861, append some nit changes
12fdb7029 is described below

commit 12fdb7029a4921ff035c8a979c1580447d0acb72
Author: Hongze Zhang 
AuthorDate: Tue May 28 11:11:19 2024 +0800

[VL] Following #5861, append some nit changes
---
 .../apache/gluten/extension/columnar/enumerated/RemoveFilter.scala| 4 ++--
 .../main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala   | 2 +-
 shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala  | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)

diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala
index a3b4831a6..55b29cd56 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveFilter.scala
@@ -54,12 +54,12 @@ object RemoveFilter extends RasRule[SparkPlan] {
 leaf(clazz(classOf[BasicScanExecTransformer]))
   ).build())
 
-  // A noop filter placeholder that indicates that all conditions are pushed 
into scan.
+  // A noop filter placeholder that indicates that all conditions are pushed 
down to scan.
   //
   // This operator has zero cost in cost model to avoid planner from choosing 
the
   // original filter-scan that doesn't have all conditions pushed down to scan.
   //
-  // We cannot simplify remove the filter to let planner choose the scan since 
by vanilla
+  // We cannot simply remove the filter to let planner choose the pushed scan 
since by vanilla
   // Spark's definition the filter may have different output nullability than 
scan. So
   // we have to keep this empty filter to let the optimized tree have the 
identical output schema
   // with the original tree. If we simply remove the filter, possible UBs 
might be caused. For
diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala
index 4b5fa6803..c45314a9f 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala
@@ -80,7 +80,7 @@ object GlutenCostModel extends Logging {
   // by vanilla Spark and was generated by strategy 
"JoinSelectionOverrides"
   infLongCost
 case _: RemoveFilter.NoopFilter =>
-  // To make planner  choose the tree that has applied rule 
PushFilterToScan.
+  // To make planner choose the tree that has applied rule 
PushFilterToScan.
   0L
 case ColumnarToRowExec(child) => 3L
 case RowToColumnarExec(child) => 3L
diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala 
b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala
index 9db063a66..d353c75c3 100644
--- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala
+++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala
@@ -1209,7 +1209,7 @@ object GlutenConfig {
   val RAS_COST_MODEL =
 buildConf("spark.gluten.ras.costModel")
   .doc(
-"Experimental: The classpath of user-defined cost model that will be 
used by RAS. " +
+"Experimental: The class name of user-defined cost model that will be 
used by RAS. " +
   "If not specified, a rough built-in cost model will be used.")
   .stringConf
   .createWithDefaultString("rough")


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [CORE] Remove static modifier on TreeMemoryConsumers.Factory.map (#5849)

2024-05-23 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new ff84bcd8f [CORE] Remove static modifier on 
TreeMemoryConsumers.Factory.map (#5849)
ff84bcd8f is described below

commit ff84bcd8f2e4bbbe234c6c36985d2705b267a8d1
Author: Hongze Zhang 
AuthorDate: Fri May 24 09:19:55 2024 +0800

[CORE] Remove static modifier on TreeMemoryConsumers.Factory.map (#5849)
---
 dev/ci-velox-buildstatic.sh|   2 +-
 .../memtarget/spark/TreeMemoryConsumers.java   |  10 +-
 .../memtarget/spark/TreeMemoryConsumerTest.java| 122 +
 3 files changed, 127 insertions(+), 7 deletions(-)

diff --git a/dev/ci-velox-buildstatic.sh b/dev/ci-velox-buildstatic.sh
index 208490d1c..075440816 100755
--- a/dev/ci-velox-buildstatic.sh
+++ b/dev/ci-velox-buildstatic.sh
@@ -2,7 +2,7 @@ yum install sudo patch java-1.8.0-openjdk-devel -y
 cd $GITHUB_WORKSPACE/ep/build-velox/src
 ./get_velox.sh
 source /opt/rh/devtoolset-9/enable
-source /opt/gluten/dev/vcpkg/env.sh
+source $GITHUB_WORKSPACE/dev/vcpkg/env.sh
 cd $GITHUB_WORKSPACE/
 sed -i '/^headers/d' ep/build-velox/build/velox_ep/CMakeLists.txt
 export NUM_THREADS=4
diff --git 
a/gluten-core/src/main/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumers.java
 
b/gluten-core/src/main/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumers.java
index 46257d80e..1da23d15e 100644
--- 
a/gluten-core/src/main/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumers.java
+++ 
b/gluten-core/src/main/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumers.java
@@ -30,7 +30,6 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 public final class TreeMemoryConsumers {
-
   private static final Map FACTORIES = new 
ConcurrentHashMap<>();
 
   private TreeMemoryConsumers() {}
@@ -61,8 +60,7 @@ public final class TreeMemoryConsumers {
   }
 
   public static class Factory {
-
-private static final ReferenceMap MAP = new 
ReferenceMap(ReferenceMap.WEAK, ReferenceMap.WEAK);
+private final ReferenceMap map = new ReferenceMap(ReferenceMap.WEAK, 
ReferenceMap.WEAK);
 private final long perTaskCapacity;
 
 private Factory(long perTaskCapacity) {
@@ -71,9 +69,9 @@ public final class TreeMemoryConsumers {
 
 @SuppressWarnings("unchecked")
 private TreeMemoryTarget getSharedAccount(TaskMemoryManager tmm) {
-  synchronized (MAP) {
+  synchronized (map) {
 return (TreeMemoryTarget)
-MAP.computeIfAbsent(
+map.computeIfAbsent(
 tmm,
 m -> {
   TreeMemoryTarget tmc = new 
TreeMemoryConsumer((TaskMemoryManager) m);
@@ -88,7 +86,7 @@ public final class TreeMemoryConsumers {
 String name,
 List spillers,
 Map virtualChildren) {
-  TreeMemoryTarget account = getSharedAccount(tmm);
+  final TreeMemoryTarget account = getSharedAccount(tmm);
   return account.newChild(
   name, TreeMemoryConsumer.CAPACITY_UNLIMITED, spillers, 
virtualChildren);
 }
diff --git 
a/gluten-core/src/test/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumerTest.java
 
b/gluten-core/src/test/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumerTest.java
new file mode 100644
index 0..e26765d33
--- /dev/null
+++ 
b/gluten-core/src/test/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumerTest.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gluten.memory.memtarget.spark;
+
+import org.apache.gluten.GlutenConfig;
+import org.apache.gluten.memory.memtarget.TreeMemoryTarget;
+
+import org.apache.spark.TaskContext;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.util.TaskResources$;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+
+import scala.Function0;
+
+public class TreeMemoryConsumerTest {
+  @Test
+  public void testIsolated() 

(incubator-gluten) branch main updated: [VL] RAS: Optimize offload rule code to gain better compatibility with rewrite rules (#5836)

2024-05-22 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 90961bc90 [VL] RAS: Optimize offload rule code to gain better 
compatibility with rewrite rules (#5836)
90961bc90 is described below

commit 90961bc907955409e1f3b7c09af00aa3bf7abf16
Author: Hongze Zhang 
AuthorDate: Thu May 23 08:45:33 2024 +0800

[VL] RAS: Optimize offload rule code to gain better compatibility with 
rewrite rules (#5836)
---
 .../columnar/enumerated/EnumeratedTransform.scala  |  59 ++---
 .../columnar/enumerated/PushFilterToScan.scala |   6 +-
 .../extension/columnar/enumerated/RasOffload.scala | 147 +++--
 .../columnar/enumerated/RasOffloadFilter.scala |   5 +-
 ...gregate.scala => RasOffloadHashAggregate.scala} |   7 +-
 .../columnar/enumerated/RemoveFilter.scala |   2 +-
 .../extension/columnar/transition/Transition.scala |   2 +-
 .../scala/org/apache/gluten/ras/path/Pattern.scala |  22 ++-
 .../org/apache/gluten/ras/path/WizardSuite.scala   |  21 ++-
 .../org/apache/gluten/ras/rule/PatternSuite.scala  |  37 +++---
 10 files changed, 206 insertions(+), 102 deletions(-)

diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala
index 50f0dce13..c41c1ca2c 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala
@@ -16,21 +16,29 @@
  */
 package org.apache.gluten.extension.columnar.enumerated
 
-import org.apache.gluten.extension.columnar.{OffloadExchange, OffloadJoin, 
OffloadOthers, OffloadSingleNode}
+import org.apache.gluten.extension.columnar.{OffloadExchange, OffloadJoin, 
OffloadOthers}
 import org.apache.gluten.extension.columnar.transition.ConventionReq
 import org.apache.gluten.planner.GlutenOptimization
 import org.apache.gluten.planner.property.Conv
 import org.apache.gluten.ras.property.PropertySet
+import org.apache.gluten.sql.shims.SparkShimLoader
 import org.apache.gluten.utils.LogLevelUtil
 
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.aggregate.{ObjectHashAggregateExec, 
SortAggregateExec}
+import org.apache.spark.sql.execution.datasources.WriteFilesExec
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase
+import org.apache.spark.sql.execution.exchange.Exchange
+import org.apache.spark.sql.execution.joins.BaseJoinExec
+import org.apache.spark.sql.execution.python.EvalPythonExec
+import org.apache.spark.sql.execution.window.WindowExec
+import org.apache.spark.sql.hive.HiveTableScanExecTransformer
 
 case class EnumeratedTransform(session: SparkSession, outputsColumnar: Boolean)
   extends Rule[SparkPlan]
   with LogLevelUtil {
-  import EnumeratedTransform._
 
   private val rules = List(
 new PushFilterToScan(RasOffload.validator),
@@ -40,11 +48,35 @@ case class EnumeratedTransform(session: SparkSession, 
outputsColumnar: Boolean)
   // TODO: Should obey ReplaceSingleNode#applyScanNotTransformable to select
   //  (vanilla) scan with cheaper sub-query plan through cost model.
   private val offloadRules = List(
-new AsRasOffload(OffloadOthers()),
-new AsRasOffload(OffloadExchange()),
-new AsRasOffload(OffloadJoin()),
-RasOffloadAggregate,
-RasOffloadFilter
+RasOffload.from[Exchange](OffloadExchange()).toRule,
+RasOffload.from[BaseJoinExec](OffloadJoin()).toRule,
+RasOffloadHashAggregate.toRule,
+RasOffloadFilter.toRule,
+RasOffload.from[DataSourceV2ScanExecBase](OffloadOthers()).toRule,
+RasOffload.from[DataSourceScanExec](OffloadOthers()).toRule,
+RasOffload
+  .from(
+(node: SparkPlan) => 
HiveTableScanExecTransformer.isHiveTableScan(node),
+OffloadOthers())
+  .toRule,
+RasOffload.from[CoalesceExec](OffloadOthers()).toRule,
+RasOffload.from[ProjectExec](OffloadOthers()).toRule,
+RasOffload.from[SortAggregateExec](OffloadOthers()).toRule,
+RasOffload.from[ObjectHashAggregateExec](OffloadOthers()).toRule,
+RasOffload.from[UnionExec](OffloadOthers()).toRule,
+RasOffload.from[ExpandExec](OffloadOthers()).toRule,
+RasOffload.from[WriteFilesExec](OffloadOthers()).toRule,
+RasOffload.from[SortExec](OffloadOthers()).toRule,
+RasOffload.from[TakeOrderedAndProjectExec](OffloadOthers()).toRule,
+RasOffload.from[WindowExec](OffloadOthers()).toRule,
+RasOffload
+  .from(
+(node: Spa

(incubator-gluten) branch main updated: [VL] RAS: Reuse same code path with heuristic planner for convention enforcement (#5824)

2024-05-22 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new d6b298221 [VL] RAS: Reuse same code path with heuristic planner for 
convention enforcement (#5824)
d6b298221 is described below

commit d6b298221f1360626e52862985f78abc7436183d
Author: Hongze Zhang 
AuthorDate: Wed May 22 17:55:00 2024 +0800

[VL] RAS: Reuse same code path with heuristic planner for convention 
enforcement (#5824)
---
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  |   3 +-
 .../org/apache/gluten/planner/VeloxRasSuite.scala  |  15 ++-
 .../gluten/execution/ColumnarToRowExecBase.scala   |  10 +-
 .../org/apache/gluten/extension/GlutenPlan.scala   |   3 +-
 .../columnar/enumerated/EnumeratedTransform.scala  |  13 +-
 .../extension/columnar/transition/Convention.scala |   4 +
 .../columnar/transition/ConventionFunc.scala   | 115 +---
 .../columnar/transition/ConventionReq.scala|  17 ++-
 .../extension/columnar/transition/Transition.scala |  30 -
 .../columnar/transition/Transitions.scala  |  48 +--
 .../gluten/planner/plan/GlutenPlanModel.scala  |  70 +++---
 .../org/apache/gluten/planner/property/Conv.scala  | 106 +++
 .../gluten/planner/property/Convention.scala   | 147 -
 .../planner/property/GlutenPropertyModel.scala |   6 +-
 .../org/apache/spark/util/SparkTaskUtil.scala  |  21 ++-
 .../apache/gluten/columnarbatch/ArrowBatch.scala   |   7 +-
 .../scala/org/apache/gluten/ras/PlanModel.scala|   2 +-
 .../src/main/scala/org/apache/gluten/ras/Ras.scala |  22 ++-
 .../scala/org/apache/gluten/ras/RasGroup.scala |  10 +-
 .../main/scala/org/apache/gluten/ras/RasNode.scala |   6 +-
 .../gluten/ras/exaustive/ExhaustivePlanner.scala   |   2 +-
 .../org/apache/gluten/ras/rule/RuleApplier.scala   |   2 +-
 .../apache/gluten/ras/vis/GraphvizVisualizer.scala |   2 +-
 .../org/apache/gluten/ras/OperationSuite.scala |   7 +-
 .../org/apache/gluten/ras/PropertySuite.scala  |   9 +-
 .../scala/org/apache/gluten/ras/RasSuiteBase.scala |   8 +-
 .../org/apache/gluten/ras/mock/MockMemoState.scala |   2 +-
 .../org/apache/gluten/ras/mock/MockRasPath.scala   |   2 +-
 .../gluten/ras/specific/DistributedSuite.scala |  18 +--
 29 files changed, 401 insertions(+), 306 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
index 2d37b1185..322116582 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
@@ -81,7 +81,8 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
*/
   override def batchTypeFunc(): BatchOverride = {
 case i: InMemoryTableScanExec
-if 
i.relation.cacheBuilder.serializer.isInstanceOf[ColumnarCachedBatchSerializer] 
=>
+if i.supportsColumnar && i.relation.cacheBuilder.serializer
+  .isInstanceOf[ColumnarCachedBatchSerializer] =>
   VeloxBatch
   }
 
diff --git 
a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala 
b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
index 4690ef516..ae2cea0ba 100644
--- 
a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
+++ 
b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
@@ -16,7 +16,8 @@
  */
 package org.apache.gluten.planner
 
-import org.apache.gluten.planner.property.Conventions
+import org.apache.gluten.extension.columnar.transition.ConventionReq
+import org.apache.gluten.planner.property.Conv
 import org.apache.gluten.ras.Best.BestNotFoundException
 import org.apache.gluten.ras.Ras
 import org.apache.gluten.ras.RasSuiteBase._
@@ -44,7 +45,7 @@ class VeloxRasSuite extends SharedSparkSession {
   test("C2R, R2C - explicitly requires any properties") {
 val in = RowUnary(RowLeaf(TRIVIAL_SCHEMA))
 val planner =
-  newRas().newPlanner(in, PropertySet(List(Conventions.ANY)))
+  newRas().newPlanner(in, PropertySet(List(Conv.any)))
 val out = planner.plan()
 assert(out == RowUnary(RowLeaf(TRIVIAL_SCHEMA)))
   }
@@ -52,7 +53,7 @@ class VeloxRasSuite extends SharedSparkSession {
   test("C2R, R2C - requires columnar output") {
 val in = RowUnary(RowLeaf(TRIVIAL_SCHEMA))
 val planner =
-  newRas().newPlanner(in, PropertySet(List(Conventions.VANILLA_COLUMNAR)))
+  newRas().newPlanner(in, 
PropertySet(List(Conv.req(ConventionReq.vanillaBatch
 val out = planner.plan()
 assert(out == RowToColumnarExec(RowUnary(RowLeaf(TRIVIAL_SCHEMA
   }
@@ 

(incubator-gluten) branch main updated: [GLUTEN-5837][VL] Fix duplicated projection name during substrait GenerateRel conversion (#5838)

2024-05-22 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new d966ea7f4 [GLUTEN-5837][VL] Fix duplicated projection name during 
substrait GenerateRel conversion (#5838)
d966ea7f4 is described below

commit d966ea7f4ad86aa4a974f3fa71269e8c2ba385e6
Author: James Xu 
AuthorDate: Wed May 22 15:20:48 2024 +0800

[GLUTEN-5837][VL] Fix duplicated projection name during substrait 
GenerateRel conversion (#5838)
---
 cpp/velox/substrait/SubstraitToVeloxPlan.cc | 7 +++
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc 
b/cpp/velox/substrait/SubstraitToVeloxPlan.cc
index c07826f5a..236203066 100644
--- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc
+++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc
@@ -794,15 +794,14 @@ core::PlanNodePtr 
SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::
 unnest.emplace_back(unnestFieldExpr);
   }
 
-  // TODO(yuan): get from generator output
   std::vector unnestNames;
   int unnestIndex = 0;
   for (const auto& variable : unnest) {
 if (variable->type()->isArray()) {
-  unnestNames.emplace_back(fmt::format("C{}", unnestIndex++));
+  unnestNames.emplace_back(SubstraitParser::makeNodeName(planNodeId_, 
unnestIndex++));
 } else if (variable->type()->isMap()) {
-  unnestNames.emplace_back(fmt::format("C{}", unnestIndex++));
-  unnestNames.emplace_back(fmt::format("C{}", unnestIndex++));
+  unnestNames.emplace_back(SubstraitParser::makeNodeName(planNodeId_, 
unnestIndex++));
+  unnestNames.emplace_back(SubstraitParser::makeNodeName(planNodeId_, 
unnestIndex++));
 } else {
   VELOX_FAIL(
   "Unexpected type of unnest variable. Expected ARRAY or MAP, but got 
{}.", variable->type()->toString());


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [GLUTEN-5773][VL] Update aws-sdk-cpp version to 1.11.285 (from 1.11.169) (#5774)

2024-05-20 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new ce92b805f [GLUTEN-5773][VL] Update aws-sdk-cpp version to 1.11.285 
(from 1.11.169) (#5774)
ce92b805f is described below

commit ce92b805fe6fc34c463b40dee42f5a7b83525e6c
Author: Yan Ma 
AuthorDate: Tue May 21 12:42:51 2024 +0800

[GLUTEN-5773][VL] Update aws-sdk-cpp version to 1.11.285 (from 1.11.169) 
(#5774)
---
 dev/vcpkg/ports/aws-sdk-cpp/fix-header.patch| 12 
 dev/vcpkg/ports/aws-sdk-cpp/fix_find_curl.patch | 31 +++
 dev/vcpkg/ports/aws-sdk-cpp/portfile.cmake  | 18 +++---
 dev/vcpkg/ports/aws-sdk-cpp/vcpkg.in.json   |  2 +-
 dev/vcpkg/ports/aws-sdk-cpp/vcpkg.json  | 73 ++---
 5 files changed, 107 insertions(+), 29 deletions(-)

diff --git a/dev/vcpkg/ports/aws-sdk-cpp/fix-header.patch 
b/dev/vcpkg/ports/aws-sdk-cpp/fix-header.patch
deleted file mode 100644
index be4511ada..0
--- a/dev/vcpkg/ports/aws-sdk-cpp/fix-header.patch
+++ /dev/null
@@ -1,12 +0,0 @@
-diff --git a/src/aws-cpp-sdk-core/include/aws/core/Aws.h 
b/src/aws-cpp-sdk-core/include/aws/core/Aws.h
-index 5c27e75a84c..d221af2039b 100644
 a/src/aws-cpp-sdk-core/include/aws/core/Aws.h
-+++ b/src/aws-cpp-sdk-core/include/aws/core/Aws.h
-@@ -12,6 +12,7 @@
- #include 
- #include 
- #include 
-+#include 
- #include 
- #include 
- 
diff --git a/dev/vcpkg/ports/aws-sdk-cpp/fix_find_curl.patch 
b/dev/vcpkg/ports/aws-sdk-cpp/fix_find_curl.patch
new file mode 100644
index 0..6f32da025
--- /dev/null
+++ b/dev/vcpkg/ports/aws-sdk-cpp/fix_find_curl.patch
@@ -0,0 +1,31 @@
+diff --git a/cmake/external_dependencies.cmake 
b/cmake/external_dependencies.cmake
+index acf16c0..3a49fb4 100644
+--- a/cmake/external_dependencies.cmake
 b/cmake/external_dependencies.cmake
+@@ -80,23 +80,12 @@ if(NOT NO_HTTP_CLIENT AND NOT USE_CRT_HTTP_CLIENT)
+ set(BUILD_CURL 1)
+ message(STATUS "  Building Curl as part of AWS SDK")
+ else()
+-include(FindCURL)
++find_package(CURL REQUIRED)
+ if(NOT CURL_FOUND)
+ message(FATAL_ERROR "Could not find curl")
++else()
++message(STATUS "  Curl library: ${CURL_LIBRARIES}")
+ endif()
+-
+-# When built from source using cmake, curl does not include
+-# CURL_INCLUDE_DIRS or CURL_INCLUDE_DIRS so we need to use
+-# find_package to fix it
+-if ("${CURL_INCLUDE_DIRS}" STREQUAL "" AND "${CURL_LIBRARIES}" 
STREQUAL "")
+-message(STATUS "Could not find curl include or library path, 
falling back to find with config.")
+-find_package(CURL)
+-set(CURL_LIBRARIES CURL::libcurl)
+-else ()
+-message(STATUS "  Curl include directory: 
${CURL_INCLUDE_DIRS}")
+-List(APPEND EXTERNAL_DEPS_INCLUDE_DIRS ${CURL_INCLUDE_DIRS})
+-set(CLIENT_LIBS ${CURL_LIBRARIES})
+-endif ()
+ set(CLIENT_LIBS_ABSTRACT_NAME curl)
+ message(STATUS "  Curl target link: ${CURL_LIBRARIES}")
+ endif()
diff --git a/dev/vcpkg/ports/aws-sdk-cpp/portfile.cmake 
b/dev/vcpkg/ports/aws-sdk-cpp/portfile.cmake
index ac3f2292b..71e701486 100644
--- a/dev/vcpkg/ports/aws-sdk-cpp/portfile.cmake
+++ b/dev/vcpkg/ports/aws-sdk-cpp/portfile.cmake
@@ -4,18 +4,18 @@ vcpkg_from_github(
 OUT_SOURCE_PATH SOURCE_PATH
 REPO aws/aws-sdk-cpp
 REF "${VERSION}"
-SHA512 
63de900870e9bec23d42e9458e0e9b1579a9e2dc7b0f404eae1b0dd406898b6d6841c5e2f498710b3828f212705437da3a2fe94813a6c3a842945100a05ae368
+SHA512 
826be806ddd87eb452f97df70b19df4194e984775408d8f99246244b6949abcab583e4cbe1ae3bc5d61f3c78267d0e75ea9e69956188ab12e0318344a4314591
 PATCHES
 patch-relocatable-rpath.patch
 fix-aws-root.patch
 lock-curl-http-and-tls-settings.patch
 fix-awsmigrationhub-build.patch
-fix-header.patch
+fix_find_curl.patch
 )
 
 string(COMPARE EQUAL "${VCPKG_CRT_LINKAGE}" "dynamic" FORCE_SHARED_CRT)
 
-set(EXTRA_ARGS)
+set(EXTRA_ARGS "")
 if(VCPKG_TARGET_IS_OSX OR VCPKG_TARGET_IS_IOS)
 set(rpath "@loader_path")
 elseif (VCPKG_TARGET_IS_ANDROID)
@@ -31,6 +31,7 @@ else()
 set(rpath "\$ORIGIN")
 endif()
 
+string(REPLACE "awsmigrationhub" "AWSMigrationHub" targets "${FEATURES}")
 vcpkg_cmake_configure(
 SOURCE_PATH "${SOURCE_PATH}"
 DISABLE_PARALLEL_CONFIGURE
@@ -39,7 +40,7 @@ vcpkg_cmake_configure(
 "-DENABLE_UNITY_BUILD=ON"
 "-DENABLE_TESTING=OFF"
 "-DF

(incubator-gluten) branch main updated: [VL][Minor] Fix warnings caused by -Wunused-but-set-variable

2024-05-20 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 6195e5d67 [VL][Minor] Fix warnings caused by -Wunused-but-set-variable
6195e5d67 is described below

commit 6195e5d67e010d6f7d05921bc77f52a6d1349472
Author: zhaokuo 
AuthorDate: Mon May 20 20:19:59 2024 +0800

[VL][Minor] Fix warnings caused by -Wunused-but-set-variable
---
 cpp/velox/substrait/SubstraitToVeloxPlan.cc | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc 
b/cpp/velox/substrait/SubstraitToVeloxPlan.cc
index b50f9bd34..34ba6057c 100644
--- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc
+++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc
@@ -2158,8 +2158,8 @@ void 
SubstraitToVeloxPlanConverter::constructSubfieldFilters(
   upperBound = getMax();
 }
 
-bool lowerUnbounded = true;
-bool upperUnbounded = true;
+[[maybe_unused]] bool lowerUnbounded = true;
+[[maybe_unused]] bool upperUnbounded = true;
 bool lowerExclusive = false;
 bool upperExclusive = false;
 


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Move memory reservation block computation logic into AllocationListener

2024-05-20 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 4a951ba78 [VL] Move memory reservation block computation logic into 
AllocationListener
4a951ba78 is described below

commit 4a951ba78f07926cb1a85429fa43584eba38be44
Author: Yang Zhang 
AuthorDate: Mon May 20 19:23:38 2024 +0800

[VL] Move memory reservation block computation logic into AllocationListener
---
 cpp/core/config/GlutenConfig.h |   3 +
 cpp/core/jni/JniCommon.h   |  59 +++---
 cpp/core/jni/JniWrapper.cc |   5 +-
 cpp/core/memory/AllocationListener.h   |  43 +++
 cpp/core/memory/MemoryAllocator.cc |  55 +++--
 cpp/core/memory/MemoryAllocator.h  |   5 +-
 cpp/velox/benchmarks/common/BenchmarkUtils.cc  |   3 +-
 cpp/velox/memory/VeloxMemoryManager.cc |  17 ++-
 cpp/velox/memory/VeloxMemoryManager.h  |   8 +-
 cpp/velox/tests/CMakeLists.txt |  14 +--
 cpp/velox/tests/FunctionTest.cc|  19 ++-
 cpp/velox/tests/MemoryManagerTest.cc   | 129 +
 .../tests/Substrait2VeloxPlanConversionTest.cc |   3 +-
 .../tests/Substrait2VeloxPlanValidatorTest.cc  |   3 -
 .../tests/VeloxColumnarBatchSerializerTest.cc  |  10 +-
 cpp/velox/tests/VeloxColumnarBatchTest.cc  |   7 +-
 cpp/velox/tests/VeloxColumnarToRowTest.cc  |   6 +-
 cpp/velox/tests/VeloxRowToColumnarTest.cc  |   3 -
 .../gluten/memory/nmm/NativeMemoryManager.java |  13 +--
 19 files changed, 261 insertions(+), 144 deletions(-)

diff --git a/cpp/core/config/GlutenConfig.h b/cpp/core/config/GlutenConfig.h
index 16a18f6be..a039537b7 100644
--- a/cpp/core/config/GlutenConfig.h
+++ b/cpp/core/config/GlutenConfig.h
@@ -42,6 +42,9 @@ const std::string kSparkOffHeapMemory = 
"spark.gluten.memory.offHeap.size.in.byt
 
 const std::string kSparkTaskOffHeapMemory = 
"spark.gluten.memory.task.offHeap.size.in.bytes";
 
+const std::string kMemoryReservationBlockSize = 
"spark.gluten.memory.reservationBlockSize";
+const uint64_t kMemoryReservationBlockSizeDefault = 8 << 20;
+
 const std::string kSparkBatchSize = "spark.gluten.sql.columnar.maxBatchSize";
 
 const std::string kParquetBlockSize = "parquet.block.size";
diff --git a/cpp/core/jni/JniCommon.h b/cpp/core/jni/JniCommon.h
index aa3b2b884..5858a70e9 100644
--- a/cpp/core/jni/JniCommon.h
+++ b/cpp/core/jni/JniCommon.h
@@ -322,13 +322,8 @@ static inline gluten::CompressionMode 
getCompressionMode(JNIEnv* env, jstring co
 
 class SparkAllocationListener final : public gluten::AllocationListener {
  public:
-  SparkAllocationListener(
-  JavaVM* vm,
-  jobject jListenerLocalRef,
-  jmethodID jReserveMethod,
-  jmethodID jUnreserveMethod,
-  int64_t blockSize)
-  : vm_(vm), jReserveMethod_(jReserveMethod), 
jUnreserveMethod_(jUnreserveMethod), blockSize_(blockSize) {
+  SparkAllocationListener(JavaVM* vm, jobject jListenerLocalRef, jmethodID 
jReserveMethod, jmethodID jUnreserveMethod)
+  : vm_(vm), jReserveMethod_(jReserveMethod), 
jUnreserveMethod_(jUnreserveMethod) {
 JNIEnv* env;
 attachCurrentThreadAsDaemonOrThrow(vm_, );
 jListenerGlobalRef_ = env->NewGlobalRef(jListenerLocalRef);
@@ -350,7 +345,20 @@ class SparkAllocationListener final : public 
gluten::AllocationListener {
   }
 
   void allocationChanged(int64_t size) override {
-updateReservation(size);
+if (size == 0) {
+  return;
+}
+JNIEnv* env;
+attachCurrentThreadAsDaemonOrThrow(vm_, );
+if (size < 0) {
+  env->CallLongMethod(jListenerGlobalRef_, jUnreserveMethod_, -size);
+  checkException(env);
+} else {
+  env->CallLongMethod(jListenerGlobalRef_, jReserveMethod_, size);
+  checkException(env);
+}
+bytesReserved_ += size;
+maxBytesReserved_ = std::max(bytesReserved_, maxBytesReserved_);
   }
 
   int64_t currentBytes() override {
@@ -362,47 +370,12 @@ class SparkAllocationListener final : public 
gluten::AllocationListener {
   }
 
  private:
-  int64_t reserve(int64_t diff) {
-std::lock_guard lock(mutex_);
-bytesReserved_ += diff;
-int64_t newBlockCount;
-if (bytesReserved_ == 0) {
-  newBlockCount = 0;
-} else {
-  // ceil to get the required block number
-  newBlockCount = (bytesReserved_ - 1) / blockSize_ + 1;
-}
-int64_t bytesGranted = (newBlockCount - blocksReserved_) * blockSize_;
-blocksReserved_ = newBlockCount;
-maxBytesReserved_ = std::max(maxBytesReserved_, bytesReserved_);
-return bytesGranted;
-  }
-
-  void updateReservation(int64_t diff) {
-int64_t granted = reserve(diff);

(incubator-gluten) branch main updated: [CORE] Rework planner C2R / R2C code with new transition facilities (#5767)

2024-05-20 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 10182a52d [CORE] Rework planner C2R / R2C code with new transition 
facilities (#5767)
10182a52d is described below

commit 10182a52d409cb659169cbfe1d7f1869d9205dce
Author: Hongze Zhang 
AuthorDate: Mon May 20 15:46:39 2024 +0800

[CORE] Rework planner C2R / R2C code with new transition facilities (#5767)
---
 .../clickhouse/CHSparkPlanExecApi.scala|  24 +--
 .../gluten/backendsapi/clickhouse/package.scala|  37 
 ...nClickHouseTPCDSParquetGraceHashJoinSuite.scala |   2 +-
 ...nClickHouseTPCDSParquetSortMergeJoinSuite.scala |   2 +-
 .../GlutenClickHouseTPCDSParquetSuite.scala|   2 +-
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  |  50 ++---
 .../apache/gluten/backendsapi/velox/package.scala  |  43 
 .../datasource/v2/ArrowBatchScanExec.scala |   6 +
 .../python/ColumnarArrowEvalPythonExec.scala   |   2 +-
 .../sql/execution/ArrowFileSourceScanExec.scala|   6 +
 .../sql/execution/VeloxParquetWriteSuite.scala |   2 +-
 .../gluten/backendsapi/SparkPlanExecApi.scala  |  21 +-
 .../gluten/expression/ExpressionConverter.scala|  19 +-
 .../gluten/extension/ColumnarOverrides.scala   |   3 +-
 .../org/apache/gluten/extension/GlutenPlan.scala   |  17 +-
 .../extension/columnar/ColumnarTransitions.scala   | 109 --
 .../extension/columnar/ExpandFallbackPolicy.scala  |  15 +-
 .../extension/columnar/MiscColumnarRules.scala |  32 +--
 .../columnar/enumerated/EnumeratedApplier.scala|   8 +-
 .../columnar/enumerated/PushFilterToScan.scala |   6 +
 .../columnar/heuristic/HeuristicApplier.scala  |   8 +-
 .../extension/columnar/transition/Convention.scala | 113 ++
 .../columnar/transition/ConventionFunc.scala   | 115 ++
 .../columnar/transition/ConventionReq.scala|  54 +
 .../extension/columnar/transition/Transition.scala | 186 
 .../columnar/transition/Transitions.scala  | 164 +++
 .../extension/columnar/transition/package.scala|  58 +
 .../gluten/planner/cost/GlutenCostModel.scala  |   7 +-
 .../gluten/planner/property/Convention.scala   |  26 +--
 .../scala/org/apache/gluten/utils/PlanUtil.scala   |  46 +---
 .../ColumnarCollapseTransformStages.scala  |  15 +-
 .../datasources/GlutenWriterColumnarRules.scala|   3 +-
 .../execution/WholeStageTransformerSuite.scala |   3 +-
 .../columnar/transition/TransitionSuite.scala  | 234 +
 .../org/apache/gluten/test}/FallbackUtil.scala |   8 +-
 .../apache/gluten/columnarbatch/ArrowBatch.scala   |  41 
 gluten-ut/pom.xml  |   7 +
 .../sql/execution/FallbackStrategiesSuite.scala|   2 +-
 .../benchmarks/ParquetReadBenchmark.scala  |   5 +-
 .../spark/sql/GlutenStringFunctionsSuite.scala |   2 +-
 .../sql/execution/FallbackStrategiesSuite.scala|   3 +-
 .../benchmarks/ParquetReadBenchmark.scala  |   5 +-
 .../spark/sql/GlutenStringFunctionsSuite.scala |   2 +-
 .../sql/execution/FallbackStrategiesSuite.scala|   3 +-
 .../benchmarks/ParquetReadBenchmark.scala  |   4 +-
 .../spark/sql/GlutenStringFunctionsSuite.scala |   2 +-
 .../sql/execution/FallbackStrategiesSuite.scala|   3 +-
 .../benchmarks/ParquetReadBenchmark.scala  |   5 +-
 .../org/apache/gluten/sql/shims/SparkShims.scala   |   2 +
 .../gluten/sql/shims/spark34/Spark34Shims.scala|   6 +
 .../gluten/sql/shims/spark35/Spark35Shims.scala|   8 +-
 51 files changed, 1212 insertions(+), 334 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 465041621..cb706d817 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -25,6 +25,7 @@ import 
org.apache.gluten.expression.ConverterUtils.FunctionConfig
 import org.apache.gluten.extension.{CountDistinctWithoutExpand, 
FallbackBroadcastHashJoin, FallbackBroadcastHashJoinPrepQueryStage, 
RewriteToDateExpresstionRule}
 import org.apache.gluten.extension.columnar.AddTransformHintRule
 import 
org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides
+import org.apache.gluten.extension.columnar.transition.Convention
 import org.apache.gluten.sql.shims.SparkShimLoader
 import org.apache.gluten.substrait.expression.{ExpressionBuilder, 
ExpressionNode, WindowFunctionNode}
 import org.apache.gluten.utils.CHJoinValidateUtil
@@ -71,6 +72,9 @@ import

(incubator-gluten) 01/01: [CORE] ASF repo config: Set required_signatures to false

2024-05-20 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch zhztheplayer-patch-1
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git

commit c2f4cfed4bfb3bc6a78d3edac53bfd167e7ac993
Author: Hongze Zhang 
AuthorDate: Mon May 20 15:35:33 2024 +0800

[CORE] ASF repo config: Set required_signatures to false
---
 .asf.yaml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.asf.yaml b/.asf.yaml
index cfe3edf0c..ae4827046 100644
--- a/.asf.yaml
+++ b/.asf.yaml
@@ -36,7 +36,7 @@ github:
   required_pull_request_reviews:
 dismiss_stale_reviews: true
 required_approving_review_count: 1
-  required_signatures: true
+  required_signatures: false
   required_linear_history: true
   required_conversation_resolution: true
   features:


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch zhztheplayer-patch-1 created (now c2f4cfed4)

2024-05-20 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch zhztheplayer-patch-1
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


  at c2f4cfed4 [CORE] ASF repo config: Set required_signatures to false

This branch includes the following new commits:

 new c2f4cfed4 [CORE] ASF repo config: Set required_signatures to false

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Use MemConfig to replace MemConfigMutable to make the code cleaner and to improve performance (#5784)

2024-05-17 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 038d9cbaa [VL] Use MemConfig to replace MemConfigMutable to make the 
code cleaner and to improve performance (#5784)
038d9cbaa is described below

commit 038d9cbaa840a7c9aea5324e510dffaace4a8801
Author: zhaokuo 
AuthorDate: Fri May 17 16:56:21 2024 +0800

[VL] Use MemConfig to replace MemConfigMutable to make the code cleaner and 
to improve performance (#5784)
---
 cpp/velox/compute/VeloxBackend.cc  | 57 +-
 cpp/velox/compute/VeloxBackend.h   |  6 ++-
 cpp/velox/compute/VeloxRuntime.cc  | 12 ++---
 cpp/velox/compute/VeloxRuntime.h   |  2 +-
 cpp/velox/compute/WholeStageResultIterator.cc  |  2 +-
 cpp/velox/compute/WholeStageResultIterator.h   |  2 +-
 .../operators/writer/VeloxParquetDatasourceABFS.h  |  6 +--
 .../operators/writer/VeloxParquetDatasourceHDFS.h  |  8 +--
 .../operators/writer/VeloxParquetDatasourceS3.h|  8 +--
 cpp/velox/substrait/SubstraitToVeloxPlan.cc|  3 +-
 cpp/velox/utils/ConfigExtractor.cc | 37 +++---
 cpp/velox/utils/ConfigExtractor.h  |  3 +-
 12 files changed, 61 insertions(+), 85 deletions(-)

diff --git a/cpp/velox/compute/VeloxBackend.cc 
b/cpp/velox/compute/VeloxBackend.cc
index b2fb1c964..187c36e1e 100644
--- a/cpp/velox/compute/VeloxBackend.cc
+++ b/cpp/velox/compute/VeloxBackend.cc
@@ -61,13 +61,13 @@ gluten::Runtime* veloxRuntimeFactory(const 
std::unordered_map& 
conf) {
-  backendConf_ = 
std::make_shared(conf);
+  backendConf_ = std::make_shared(conf);
 
   // Register Velox runtime factory
   gluten::Runtime::registerFactory(gluten::kVeloxRuntimeKind, 
veloxRuntimeFactory);
 
   if (backendConf_->get(kDebugModeEnabled, false)) {
-LOG(INFO) << "VeloxBackend config:" << 
printConfig(backendConf_->valuesCopy());
+LOG(INFO) << "VeloxBackend config:" << printConfig(backendConf_->values());
   }
 
   // Init glog and log level.
@@ -188,46 +188,39 @@ void VeloxBackend::initCache() {
 
 void VeloxBackend::initConnector() {
   // The configs below are used at process level.
-  auto connectorConf = 
std::make_shared(backendConf_->valuesCopy());
+  std::unordered_map connectorConfMap = 
backendConf_->values();
 
   auto hiveConf = getHiveConfig(backendConf_);
   for (auto& [k, v] : hiveConf->valuesCopy()) {
-connectorConf->setValue(k, v);
+connectorConfMap[k] = v;
   }
 
 #ifdef ENABLE_ABFS
-  const auto& confValue = backendConf_->valuesCopy();
+  const auto& confValue = backendConf_->values();
   for (auto& [k, v] : confValue) {
 if (k.find("fs.azure.account.key") == 0) {
-  connectorConf->setValue(k, v);
+  connectorConfMap[k] = v;
 } else if (k.find("spark.hadoop.fs.azure.account.key") == 0) {
   constexpr int32_t accountKeyPrefixLength = 13;
-  connectorConf->setValue(k.substr(accountKeyPrefixLength), v);
+  connectorConfMap[k.substr(accountKeyPrefixLength)] = v;
 }
   }
 #endif
-
-  connectorConf->setValue(
-  velox::connector::hive::HiveConfig::kEnableFileHandleCache,
-  backendConf_->get(kVeloxFileHandleCacheEnabled, 
kVeloxFileHandleCacheEnabledDefault) ? "true" : "false");
-
-  connectorConf->setValue(
-  velox::connector::hive::HiveConfig::kMaxCoalescedBytes,
-  backendConf_->get(kMaxCoalescedBytes, "67108864")); // 64M
-  connectorConf->setValue(
-  velox::connector::hive::HiveConfig::kMaxCoalescedDistanceBytes,
-  backendConf_->get(kMaxCoalescedDistanceBytes, "1048576")); 
// 1M
-  connectorConf->setValue(
-  velox::connector::hive::HiveConfig::kPrefetchRowGroups, 
backendConf_->get(kPrefetchRowGroups, "1"));
-  connectorConf->setValue(
-  velox::connector::hive::HiveConfig::kLoadQuantum,
-  backendConf_->get(kLoadQuantum, "268435456")); // 256M
-  connectorConf->setValue(
-  velox::connector::hive::HiveConfig::kFooterEstimatedSize,
-  backendConf_->get(kDirectorySizeGuess, "32768")); // 32K
-  connectorConf->setValue(
-  velox::connector::hive::HiveConfig::kFilePreloadThreshold,
-  backendConf_->get(kFilePreloadThreshold, "1048576")); // 1M
+  connectorConfMap[velox::connector::hive::HiveConfig::kEnableFileHandleCache] 
=
+  backendConf_->get(kVeloxFileHandleCacheEnabled, 
kVeloxFileHandleCacheEnabledDefault) ? "true" : "false";
+
+  connectorConfMap[velox::connector::hive::HiveConfig::kMaxCoalescedBytes] =
+  backendConf_->get(kMaxCoalescedBytes, "67108864"); // 64M
+  
connec

(incubator-gluten) branch main updated: [GLUTEN-5414] [VL] Support datasource v2 scan csv (#5717)

2024-05-16 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 6a110e5e6 [GLUTEN-5414] [VL] Support datasource v2 scan csv (#5717)
6a110e5e6 is described below

commit 6a110e5e60d5f195293119f42a58c9f6911c987c
Author: Jin Chengcheng 
AuthorDate: Thu May 16 16:02:54 2024 +0800

[GLUTEN-5414] [VL] Support datasource v2 scan csv (#5717)
---
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  |   2 +
 .../gluten/datasource/ArrowCSVFileFormat.scala | 277 +++--
 .../gluten/datasource/ArrowConvertorRule.scala |  49 +++-
 .../v2/ArrowCSVPartitionReaderFactory.scala| 144 +++
 .../apache/gluten/datasource/v2/ArrowCSVScan.scala |  76 ++
 .../v2/ArrowCSVScanBuilder.scala}  |  34 ++-
 .../gluten/datasource/v2/ArrowCSVTable.scala   |  68 +
 .../datasource/v2/ArrowBatchScanExec.scala |  48 
 .../gluten/extension/ArrowScanReplaceRule.scala|   7 +-
 .../org/apache/gluten/execution/TestOperator.scala |  22 +-
 .../gluten/columnarbatch/ColumnarBatches.java  |   4 +-
 .../scala/org/apache/gluten/utils/ArrowUtil.scala  |  35 +--
 .../gluten/utils/velox/VeloxTestSettings.scala |   9 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |   9 +
 .../gluten/utils/velox/VeloxTestSettings.scala |   9 +
 .../gluten/utils/velox/VeloxTestSettings.scala |   9 +
 .../execution/datasources/csv/GlutenCSVSuite.scala |   1 +
 .../datasources/v2/BatchScanExecShim.scala |   4 +
 .../datasources/v2/BatchScanExecShim.scala |   6 +
 .../datasources/v2/BatchScanExecShim.scala |   8 +
 .../datasources/v2/BatchScanExecShim.scala |   8 +
 21 files changed, 658 insertions(+), 171 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
index 33ce1ee72..f54bf9b3f 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
@@ -21,6 +21,7 @@ import org.apache.gluten.backendsapi.SparkPlanExecApi
 import org.apache.gluten.datasource.ArrowConvertorRule
 import org.apache.gluten.exception.GlutenNotSupportException
 import org.apache.gluten.execution._
+import org.apache.gluten.execution.datasource.v2.ArrowBatchScanExec
 import org.apache.gluten.expression._
 import org.apache.gluten.expression.ConverterUtils.FunctionConfig
 import org.apache.gluten.expression.aggregate.{HLLAdapter, 
VeloxBloomFilterAggregate, VeloxCollectList, VeloxCollectSet}
@@ -869,6 +870,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
 
   override def outputNativeColumnarSparkCompatibleData(plan: SparkPlan): 
Boolean = plan match {
 case _: ArrowFileSourceScanExec => true
+case _: ArrowBatchScanExec => true
 case _ => false
   }
 }
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowCSVFileFormat.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowCSVFileFormat.scala
index c05af24ff..0f6813d8f 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowCSVFileFormat.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowCSVFileFormat.scala
@@ -20,6 +20,7 @@ import org.apache.gluten.columnarbatch.ColumnarBatches
 import org.apache.gluten.exception.SchemaMismatchException
 import org.apache.gluten.execution.RowToVeloxColumnarExec
 import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
+import org.apache.gluten.memory.arrow.pool.ArrowNativeMemoryPool
 import org.apache.gluten.utils.{ArrowUtil, Iterators}
 import org.apache.gluten.vectorized.ArrowWritableColumnVector
 
@@ -41,6 +42,7 @@ import org.apache.spark.util.SerializableConfiguration
 
 import org.apache.arrow.dataset.file.FileSystemDatasetFactory
 import org.apache.arrow.dataset.scanner.ScanOptions
+import org.apache.arrow.memory.BufferAllocator
 import org.apache.arrow.vector.VectorUnloader
 import org.apache.arrow.vector.types.pojo.Schema
 import org.apache.hadoop.conf.Configuration
@@ -66,55 +68,127 @@ class ArrowCSVFileFormat extends FileFormat with 
DataSourceRegister with Logging
   sparkSession: SparkSession,
   options: Map[String, String],
   files: Seq[FileStatus]): Option[StructType] = {
-ArrowUtil.readSchema(files, fileFormat)
+ArrowUtil.readSchema(
+  files,
+  fileFormat,
+  ArrowBufferAllocators.contextInstance(),
+  ArrowNativeMemoryPool.arrowPool("infer schema"))
   }
 
   override def supportBatch(sparkSession: SparkSession, dataSchema: 
StructType): Boolean = true
 
-  private def checkHeader(
-  fi

(incubator-gluten) branch main updated: [GLUTEN-5696] Add preprojection support for ArrowEvalPythonExec (#5697)

2024-05-16 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 0cbb7f229 [GLUTEN-5696] Add preprojection support for 
ArrowEvalPythonExec (#5697)
0cbb7f229 is described below

commit 0cbb7f2297a940047dfb788caac17cb2ad540356
Author: Yan Ma 
AuthorDate: Thu May 16 15:09:44 2024 +0800

[GLUTEN-5696] Add preprojection support for ArrowEvalPythonExec (#5697)
---
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  |  8 +-
 .../python/ColumnarArrowEvalPythonExec.scala   | 97 --
 .../python/ArrowEvalPythonExecSuite.scala  | 43 +-
 .../gluten/backendsapi/SparkPlanExecApi.scala  |  4 +
 .../columnar/rewrite/PullOutPreProject.scala   |  5 ++
 .../rewrite/RewriteSparkPlanRulesManager.scala |  2 +
 6 files changed, 149 insertions(+), 10 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
index 4d41ed0c0..33ce1ee72 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
@@ -31,7 +31,7 @@ import 
org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode
 import org.apache.gluten.vectorized.{ColumnarBatchSerializer, 
ColumnarBatchSerializeResult}
 
 import org.apache.spark.{ShuffleDependency, SparkException}
-import org.apache.spark.api.python.ColumnarArrowEvalPythonExec
+import org.apache.spark.api.python.{ColumnarArrowEvalPythonExec, 
PullOutArrowEvalPythonPreProjectHelper}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.serializer.Serializer
 import org.apache.spark.shuffle.{GenShuffleWriterParameters, 
GlutenShuffleWriterWrapper}
@@ -53,6 +53,7 @@ import org.apache.spark.sql.execution.datasources.FileFormat
 import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, 
ShuffleExchangeExec}
 import org.apache.spark.sql.execution.joins.{BuildSideRelation, 
HashedRelationBroadcastMode}
 import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.python.ArrowEvalPythonExec
 import org.apache.spark.sql.execution.utils.ExecUtil
 import org.apache.spark.sql.expression.{UDFExpression, UDFResolver, 
UserDefinedAggregateFunction}
 import org.apache.spark.sql.internal.SQLConf
@@ -846,6 +847,11 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
 PullOutGenerateProjectHelper.pullOutPostProject(generate)
   }
 
+  override def genPreProjectForArrowEvalPythonExec(
+  arrowEvalPythonExec: ArrowEvalPythonExec): SparkPlan = {
+
PullOutArrowEvalPythonPreProjectHelper.pullOutPreProject(arrowEvalPythonExec)
+  }
+
   override def maybeCollapseTakeOrderedAndProject(plan: SparkPlan): SparkPlan 
= {
 // This to-top-n optimization assumes exchange operators were already 
placed in input plan.
 plan.transformUp {
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
index 77ef1c642..d3112c974 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
@@ -17,17 +17,18 @@
 package org.apache.spark.api.python
 
 import org.apache.gluten.columnarbatch.ColumnarBatches
+import org.apache.gluten.exception.GlutenException
 import org.apache.gluten.extension.GlutenPlan
 import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
-import org.apache.gluten.utils.Iterators
+import org.apache.gluten.utils.{Iterators, PullOutProjectHelper}
 import org.apache.gluten.vectorized.ArrowWritableColumnVector
 
 import org.apache.spark.{ContextAwareIterator, SparkEnv, TaskContext}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.python.{BasePythonRunnerShim, 
EvalPythonExec, PythonUDFRunner}
+import org.apache.spark.sql.execution.{ProjectExec, SparkPlan}
+import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, 
BasePythonRunnerShim, EvalPythonExec, PythonUDFRunner}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.{DataType, StructField, StructType}
 import org.apache.spark.sql.utils.{SparkArrowUtil, SparkSchemaUtil, 
SparkVectorUtil}
@@ -41,6 +42,7 @@ import java.io.{DataInputStream, DataOutputStream}
 import java.net.Socket
 import java.util.concurrent.atomic.AtomicBoolean

(incubator-gluten) branch main updated: [VL] CI: Gluten-it: Fix unreadable test reporting when there are query failures (#5753)

2024-05-14 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 3b8ea67f4 [VL] CI: Gluten-it: Fix unreadable test reporting when there 
are query failures (#5753)
3b8ea67f4 is described below

commit 3b8ea67f49d0b3afdbc327905a6cb0fefe23b56a
Author: Hongze Zhang 
AuthorDate: Wed May 15 13:56:51 2024 +0800

[VL] CI: Gluten-it: Fix unreadable test reporting when there are query 
failures (#5753)
---
 .../scala/org/apache/gluten/integration/tpc/action/Queries.scala  | 2 +-
 .../org/apache/gluten/integration/tpc/action/QueriesCompare.scala | 2 +-
 .../org/apache/gluten/integration/tpc/action/TableFormatter.scala | 8 ++--
 3 files changed, 8 insertions(+), 4 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Queries.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Queries.scala
index edeb960fc..290b8e3f5 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Queries.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Queries.scala
@@ -80,7 +80,7 @@ case class Queries(
 var all = Queries.aggregate(results, "all")
 
 if (passedCount != count) {
-  all = Queries.aggregate(succeed, "all succeed") ::: all
+  all = Queries.aggregate(succeed, "succeeded") ::: all
 }
 
 println("Overall: ")
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
index cfb3e7dc5..404d75cb4 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
@@ -80,7 +80,7 @@ case class QueriesCompare(
 var all = QueriesCompare.aggregate(results, "all")
 
 if (passedCount != count) {
-  all = QueriesCompare.aggregate(succeed, "all succeed") ::: all
+  all = QueriesCompare.aggregate(succeed, "succeeded") ::: all
 }
 
 println("Overall: ")
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/TableFormatter.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/TableFormatter.scala
index cb6ab7ebd..8aeea9938 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/TableFormatter.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/TableFormatter.scala
@@ -46,6 +46,12 @@ object TableFormatter {
 }
 
 override def print(s: OutputStream): Unit = {
+  val printer = new PrintStream(s)
+  if (rows.isEmpty) {
+printer.println("(N/A)")
+printer.flush()
+return
+  }
   val numFields = schema.fields.size
   val widths = (0 until numFields)
 .map { i =>
@@ -58,13 +64,11 @@ object TableFormatter {
 pBuilder ++= s"%${w}s|"
   }
   val pattern = pBuilder.toString()
-  val printer = new PrintStream(s)
   printer.println(String.format(pattern, schema.fields: _*))
   rows.foreach { r =>
 printer.println(String.format(pattern, r: _*))
   }
   printer.flush()
-  printer.close()
 }
   }
 


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [CORE] Add a compilation-time check to forbid case-class inheritance (#5729)

2024-05-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new ecf61ee2f [CORE] Add a compilation-time check to forbid case-class 
inheritance (#5729)
ecf61ee2f is described below

commit ecf61ee2f4bb8fefb4a48635c293cb5d7fe18fa1
Author: Hongze Zhang 
AuthorDate: Tue May 14 13:05:50 2024 +0800

[CORE] Add a compilation-time check to forbid case-class inheritance (#5729)
---
 .../org/apache/spark/sql/delta/DeltaLog.scala  | 54 +++---
 .../org/apache/spark/sql/delta/DeltaLog.scala  | 52 ++---
 .../spark/shuffle/HashPartitioningWrapper.scala|  1 +
 .../sql/delta/catalog/ClickHouseTableV2.scala  |  2 +
 .../v2/clickhouse/metadata/AddFileTags.scala   |  1 +
 pom.xml| 18 +++-
 6 files changed, 93 insertions(+), 35 deletions(-)

diff --git 
a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala
 
b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala
index 00820a006..0f6455997 100644
--- 
a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala
+++ 
b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala
@@ -81,9 +81,10 @@ class DeltaLog private (
   with SnapshotManagement
   with DeltaFileFormat
   with ReadChecksum {
-
   import org.apache.spark.sql.delta.util.FileNames._
 
+  import DeltaLog._
+
   implicit private lazy val _clock = clock
 
   protected def spark = SparkSession.active
@@ -442,8 +443,8 @@ class DeltaLog private (
 
 val fileIndex =
   TahoeLogFileIndex(spark, this, dataPath, snapshotToUse, 
partitionFilters, isTimeTravelQuery)
-var bucketSpec: Option[BucketSpec] = 
ClickHouseTableV2.getTable(this).bucketOption
-new HadoopFsRelation(
+val bucketSpec: Option[BucketSpec] = 
ClickHouseTableV2.getTable(this).bucketOption
+new DeltaHadoopFsRelation(
   fileIndex,
   partitionSchema =
 
DeltaColumnMapping.dropColumnMappingMetadata(snapshotToUse.metadata.partitionSchema),
@@ -460,20 +461,9 @@ class DeltaLog private (
   // conflict with `DeltaLog.options`.
   snapshotToUse.metadata.format.options ++ options
 )(
-  spark
-) with InsertableRelation {
-  def insert(data: DataFrame, overwrite: Boolean): Unit = {
-val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append
-WriteIntoDelta(
-  deltaLog = DeltaLog.this,
-  mode = mode,
-  new DeltaOptions(Map.empty[String, String], spark.sessionState.conf),
-  partitionColumns = Seq.empty,
-  configuration = Map.empty,
-  data = data
-).run(spark)
-  }
-}
+  spark,
+  this
+)
   }
 
   override def fileFormat(metadata: Metadata = metadata): FileFormat =
@@ -482,6 +472,36 @@ class DeltaLog private (
 }
 
 object DeltaLog extends DeltaLogging {
+  
@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
+  private class DeltaHadoopFsRelation(
+  location: FileIndex,
+  partitionSchema: StructType,
+  // The top-level columns in `dataSchema` should match the actual 
physical file schema, otherwise
+  // the ORC data source may not work with the by-ordinal mode.
+  dataSchema: StructType,
+  bucketSpec: Option[BucketSpec],
+  fileFormat: FileFormat,
+  options: Map[String, String])(sparkSession: SparkSession, deltaLog: 
DeltaLog)
+extends HadoopFsRelation(
+  location,
+  partitionSchema,
+  dataSchema,
+  bucketSpec,
+  fileFormat,
+  options)(sparkSession)
+with InsertableRelation {
+def insert(data: DataFrame, overwrite: Boolean): Unit = {
+  val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append
+  WriteIntoDelta(
+deltaLog = deltaLog,
+mode = mode,
+new DeltaOptions(Map.empty[String, String], 
sparkSession.sessionState.conf),
+partitionColumns = Seq.empty,
+configuration = Map.empty,
+data = data
+  ).run(sparkSession)
+}
+  }
 
   /**
* The key type of `DeltaLog` cache. It's a pair of the canonicalized table 
path and the file
diff --git 
a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/DeltaLog.scala
 
b/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/DeltaLog.scala
index bbf0bdc91..4cab6454d 100644
--- 
a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/DeltaLog.scala
+++ 
b/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/DeltaLog.scala
@@ -85,9 +85,10 @@ class DeltaLog private (
   with SnapshotManagement
   with DeltaFileFormat
   with ReadChecksum {
-
   import org.apache.spark.sql.delta.util.FileNames._
 
+  import DeltaLog._
+
   implicit privat

(incubator-gluten) branch main updated: [VL]: Fix VeloxColumnarWriteFilesExecwithNewChildren doesn't replace the dummy child (#5726)

2024-05-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 600a5eeb9 [VL]: Fix VeloxColumnarWriteFilesExecwithNewChildren doesn't 
replace the dummy child (#5726)
600a5eeb9 is described below

commit 600a5eeb93cf2cbd12aa2c018d28addf12510bd2
Author: Hongze Zhang 
AuthorDate: Mon May 13 18:19:46 2024 +0800

[VL]: Fix VeloxColumnarWriteFilesExecwithNewChildren doesn't replace the 
dummy child (#5726)
---
 .../execution/VeloxColumnarWriteFilesExec.scala| 61 +++---
 1 file changed, 42 insertions(+), 19 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
index 23dff990c..1d3d55afb 100644
--- 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
+++ 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
@@ -259,8 +259,9 @@ class VeloxColumnarWriteFilesRDD(
 // we need to expose a dummy child (as right child) with type "WriteFilesExec" 
to let Spark
 // choose the new write code path (version >= 3.4). The actual plan to write 
is the left child
 // of this operator.
-case class VeloxColumnarWriteFilesExec(
-child: SparkPlan,
+case class VeloxColumnarWriteFilesExec private (
+override val left: SparkPlan,
+override val right: SparkPlan,
 fileFormat: FileFormat,
 partitionColumns: Seq[Attribute],
 bucketSpec: Option[BucketSpec],
@@ -269,7 +270,8 @@ case class VeloxColumnarWriteFilesExec(
   extends BinaryExecNode
   with GlutenPlan
   with VeloxColumnarWriteFilesExec.ExecuteWriteCompatible {
-  import VeloxColumnarWriteFilesExec._
+
+  val child: SparkPlan = left
 
   override lazy val references: AttributeSet = AttributeSet.empty
 
@@ -320,28 +322,49 @@ case class VeloxColumnarWriteFilesExec(
   new VeloxColumnarWriteFilesRDD(rdd, writeFilesSpec, jobTrackerID)
 }
   }
-
-  override def left: SparkPlan = child
-
-  // This is a workaround for FileFormatWriter#write. Vanilla Spark (version 
>= 3.4) requires for
-  // a plan that has at least one node exactly of type `WriteFilesExec` that 
is a Scala case-class,
-  // to decide to choose new `#executeWrite` code path over the legacy 
`#execute` for write
-  // operation.
-  //
-  // So we add a no-op `WriteFilesExec` child to let Spark pick the new code 
path.
-  //
-  // See: FileFormatWriter#write
-  // See: V1Writes#getWriteFilesOpt
-  override val right: SparkPlan =
-WriteFilesExec(NoopLeaf(), fileFormat, partitionColumns, bucketSpec, 
options, staticPartitions)
-
   override protected def withNewChildrenInternal(
   newLeft: SparkPlan,
   newRight: SparkPlan): SparkPlan =
-copy(newLeft, fileFormat, partitionColumns, bucketSpec, options, 
staticPartitions)
+copy(newLeft, newRight, fileFormat, partitionColumns, bucketSpec, options, 
staticPartitions)
 }
 
 object VeloxColumnarWriteFilesExec {
+
+  def apply(
+  child: SparkPlan,
+  fileFormat: FileFormat,
+  partitionColumns: Seq[Attribute],
+  bucketSpec: Option[BucketSpec],
+  options: Map[String, String],
+  staticPartitions: TablePartitionSpec): VeloxColumnarWriteFilesExec = {
+// This is a workaround for FileFormatWriter#write. Vanilla Spark (version 
>= 3.4) requires for
+// a plan that has at least one node exactly of type `WriteFilesExec` that 
is a Scala
+// case-class, to decide to choose new `#executeWrite` code path over the 
legacy `#execute`
+// for write operation.
+//
+// So we add a no-op `WriteFilesExec` child to let Spark pick the new code 
path.
+//
+// See: FileFormatWriter#write
+// See: V1Writes#getWriteFilesOpt
+val right: SparkPlan =
+  WriteFilesExec(
+NoopLeaf(),
+fileFormat,
+partitionColumns,
+bucketSpec,
+options,
+staticPartitions)
+
+VeloxColumnarWriteFilesExec(
+  child,
+  right,
+  fileFormat,
+  partitionColumns,
+  bucketSpec,
+  options,
+  staticPartitions)
+  }
+
   private case class NoopLeaf() extends LeafExecNode {
 override protected def doExecute(): RDD[InternalRow] =
   throw new GlutenException(s"$nodeName does not support doExecute")


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) 01/01: Revert "[CORE] Add a compilation-time check to forbid case-class inheritance"

2024-05-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch revert-5723-wip-forbid-inherit-caseclass
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git

commit e33432bba6a2ffe48fbc8267001e660c2c18d407
Author: Hongze Zhang 
AuthorDate: Mon May 13 15:49:17 2024 +0800

Revert "[CORE] Add a compilation-time check to forbid case-class 
inheritance"

This reverts commit f5a6c98bb80b11f22a8e07abf9ee22852fc5a425.
---
 .../spark/sql/delta/catalog/ClickHouseTableV2.scala   |  2 --
 .../datasources/v2/clickhouse/metadata/AddFileTags.scala  |  1 -
 .../apache/spark/shuffle/HashPartitioningWrapper.scala|  1 -
 pom.xml   | 15 ---
 4 files changed, 19 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
index 1107c6a2e..8c09ae7c7 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
@@ -39,7 +39,6 @@ import java.{util => ju}
 
 import scala.collection.JavaConverters._
 
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class ClickHouseTableV2(
 override val spark: SparkSession,
 override val path: Path,
@@ -269,7 +268,6 @@ class ClickHouseTableV2(
   }
 }
 
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class TempClickHouseTableV2(
 override val spark: SparkSession,
 override val catalogTable: Option[CatalogTable] = None)
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
index 0680663eb..bdb3a30e9 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
@@ -28,7 +28,6 @@ import java.util.{List => JList}
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class AddMergeTreeParts(
 val database: String,
 val table: String,
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
 
b/gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
similarity index 94%
rename from 
backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
rename to 
gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
index bf1cbe4a8..06ce8fe0f 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
+++ 
b/gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
@@ -21,7 +21,6 @@ import 
org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
 
 // A wrapper for HashPartitioning to remain original hash expressions.
 // Only used by CH backend when shuffle hash expressions contains non-field 
expression.
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class HashPartitioningWrapper(
 original: Seq[Expression],
 newExpr: Seq[Expression],
diff --git a/pom.xml b/pom.xml
index a29f19832..dbf46ac17 100644
--- a/pom.xml
+++ b/pom.xml
@@ -615,20 +615,6 @@
   scala-maven-plugin
   ${scala.compiler.version}
   
-
-  
-org.wartremover
-wartremover_${scala.binary.version}
-3.1.6
-  
-
-
-  
-io.github.zhztheplayer.scalawarts
-scalawarts
-0.1.0
-  
-
 ${scala.recompile.mode}
 
   -Wconf:msg=While parsing annotations in:silent
@@ -636,7 +622,6 @@
   -Xfatal-warnings
   -deprecation
   -feature
-  
-P:wartremover:traverser:io.github.zhztheplayer.scalawarts.InheritFromCaseClass
 
   
   


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: Revert "[CORE] Add a compilation-time check to forbid case-class inheritance" (#5727)

2024-05-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 512f4e150 Revert "[CORE] Add a compilation-time check to forbid 
case-class inheritance" (#5727)
512f4e150 is described below

commit 512f4e1508d682ffa1aa64daf62551d86e06732c
Author: Hongze Zhang 
AuthorDate: Mon May 13 15:50:04 2024 +0800

Revert "[CORE] Add a compilation-time check to forbid case-class 
inheritance" (#5727)

This reverts commit f5a6c98bb80b11f22a8e07abf9ee22852fc5a425.
---
 .../spark/sql/delta/catalog/ClickHouseTableV2.scala   |  2 --
 .../datasources/v2/clickhouse/metadata/AddFileTags.scala  |  1 -
 .../apache/spark/shuffle/HashPartitioningWrapper.scala|  1 -
 pom.xml   | 15 ---
 4 files changed, 19 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
index 1107c6a2e..8c09ae7c7 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
@@ -39,7 +39,6 @@ import java.{util => ju}
 
 import scala.collection.JavaConverters._
 
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class ClickHouseTableV2(
 override val spark: SparkSession,
 override val path: Path,
@@ -269,7 +268,6 @@ class ClickHouseTableV2(
   }
 }
 
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class TempClickHouseTableV2(
 override val spark: SparkSession,
 override val catalogTable: Option[CatalogTable] = None)
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
index 0680663eb..bdb3a30e9 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
@@ -28,7 +28,6 @@ import java.util.{List => JList}
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class AddMergeTreeParts(
 val database: String,
 val table: String,
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
 
b/gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
similarity index 94%
rename from 
backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
rename to 
gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
index bf1cbe4a8..06ce8fe0f 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
+++ 
b/gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
@@ -21,7 +21,6 @@ import 
org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
 
 // A wrapper for HashPartitioning to remain original hash expressions.
 // Only used by CH backend when shuffle hash expressions contains non-field 
expression.
-@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class HashPartitioningWrapper(
 original: Seq[Expression],
 newExpr: Seq[Expression],
diff --git a/pom.xml b/pom.xml
index a29f19832..dbf46ac17 100644
--- a/pom.xml
+++ b/pom.xml
@@ -615,20 +615,6 @@
   scala-maven-plugin
   ${scala.compiler.version}
   
-
-  
-org.wartremover
-wartremover_${scala.binary.version}
-3.1.6
-  
-
-
-  
-io.github.zhztheplayer.scalawarts
-scalawarts
-0.1.0
-  
-
 ${scala.recompile.mode}
 
   -Wconf:msg=While parsing annotations in:silent
@@ -636,7 +622,6 @@
   -Xfatal-warnings
   -deprecation
   -feature
-  
-P:wartremover:traverser:io.github.zhztheplayer.scalawarts.InheritFromCaseClass
 
   
   


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch revert-5723-wip-forbid-inherit-caseclass created (now e33432bba)

2024-05-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch revert-5723-wip-forbid-inherit-caseclass
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


  at e33432bba Revert "[CORE] Add a compilation-time check to forbid 
case-class inheritance"

This branch includes the following new commits:

 new e33432bba Revert "[CORE] Add a compilation-time check to forbid 
case-class inheritance"

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [CORE] Add a compilation-time check to forbid case-class inheritance

2024-05-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new f5a6c98bb [CORE] Add a compilation-time check to forbid case-class 
inheritance
f5a6c98bb is described below

commit f5a6c98bb80b11f22a8e07abf9ee22852fc5a425
Author: Hongze Zhang 
AuthorDate: Mon May 13 15:33:21 2024 +0800

[CORE] Add a compilation-time check to forbid case-class inheritance
---
 .../apache/spark/shuffle/HashPartitioningWrapper.scala|  1 +
 .../spark/sql/delta/catalog/ClickHouseTableV2.scala   |  2 ++
 .../datasources/v2/clickhouse/metadata/AddFileTags.scala  |  1 +
 pom.xml   | 15 +++
 4 files changed, 19 insertions(+)

diff --git 
a/gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
similarity index 94%
rename from 
gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
rename to 
backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
index 06ce8fe0f..bf1cbe4a8 100644
--- 
a/gluten-core/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/HashPartitioningWrapper.scala
@@ -21,6 +21,7 @@ import 
org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
 
 // A wrapper for HashPartitioning to remain original hash expressions.
 // Only used by CH backend when shuffle hash expressions contains non-field 
expression.
+@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class HashPartitioningWrapper(
 original: Seq[Expression],
 newExpr: Seq[Expression],
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
index 8c09ae7c7..1107c6a2e 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
@@ -39,6 +39,7 @@ import java.{util => ju}
 
 import scala.collection.JavaConverters._
 
+@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class ClickHouseTableV2(
 override val spark: SparkSession,
 override val path: Path,
@@ -268,6 +269,7 @@ class ClickHouseTableV2(
   }
 }
 
+@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class TempClickHouseTableV2(
 override val spark: SparkSession,
 override val catalogTable: Option[CatalogTable] = None)
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
index bdb3a30e9..0680663eb 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
@@ -28,6 +28,7 @@ import java.util.{List => JList}
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
+@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
 class AddMergeTreeParts(
 val database: String,
 val table: String,
diff --git a/pom.xml b/pom.xml
index dbf46ac17..a29f19832 100644
--- a/pom.xml
+++ b/pom.xml
@@ -615,6 +615,20 @@
   scala-maven-plugin
   ${scala.compiler.version}
   
+
+  
+org.wartremover
+wartremover_${scala.binary.version}
+3.1.6
+  
+
+
+  
+io.github.zhztheplayer.scalawarts
+scalawarts
+0.1.0
+  
+
 ${scala.recompile.mode}
 
   -Wconf:msg=While parsing annotations in:silent
@@ -622,6 +636,7 @@
   -Xfatal-warnings
   -deprecation
   -feature
+  
-P:wartremover:traverser:io.github.zhztheplayer.scalawarts.InheritFromCaseClass
 
   
   


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [GLUTEN-5414] [VL] Move ArrowFileScanExec class to module backends-velox

2024-05-12 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 09950de2d [GLUTEN-5414] [VL] Move ArrowFileScanExec class to module 
backends-velox
09950de2d is described below

commit 09950de2dd80090a0bc0fea0631749916614cec1
Author: Jin Chengcheng 
AuthorDate: Mon May 13 09:37:36 2024 +0800

[GLUTEN-5414] [VL] Move ArrowFileScanExec class to module backends-velox
---
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  | 10 +--
 .../gluten/extension/ArrowScanReplaceRule.scala| 34 ++
 .../sql/execution/ArrowFileSourceScanExec.scala|  0
 .../gluten/backendsapi/SparkPlanExecApi.scala  |  2 ++
 .../extension/columnar/MiscColumnarRules.scala |  4 ++-
 .../extension/columnar/OffloadSingleNode.scala | 10 +--
 .../scala/org/apache/gluten/utils/PlanUtil.scala   |  6 +++-
 7 files changed, 53 insertions(+), 13 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
index 772f1cfb2..8d01ab96b 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
@@ -24,7 +24,7 @@ import org.apache.gluten.execution._
 import org.apache.gluten.expression._
 import org.apache.gluten.expression.ConverterUtils.FunctionConfig
 import org.apache.gluten.expression.aggregate.{HLLAdapter, 
VeloxBloomFilterAggregate, VeloxCollectList, VeloxCollectSet}
-import org.apache.gluten.extension.{BloomFilterMightContainJointRewriteRule, 
CollectRewriteRule, FlushableHashAggregateRule, HLLRewriteRule}
+import org.apache.gluten.extension.{ArrowScanReplaceRule, 
BloomFilterMightContainJointRewriteRule, CollectRewriteRule, 
FlushableHashAggregateRule, HLLRewriteRule}
 import org.apache.gluten.extension.columnar.TransformHints
 import org.apache.gluten.sql.shims.SparkShimLoader
 import org.apache.gluten.substrait.expression.{ExpressionBuilder, 
ExpressionNode, IfThenNode}
@@ -744,7 +744,8 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
* @return
*/
   override def genExtendedColumnarValidationRules(): List[SparkSession => 
Rule[SparkPlan]] = List(
-BloomFilterMightContainJointRewriteRule.apply
+BloomFilterMightContainJointRewriteRule.apply,
+ArrowScanReplaceRule.apply
   )
 
   /**
@@ -849,4 +850,9 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
   case other => other
 }
   }
+
+  override def outputNativeColumnarSparkCompatibleData(plan: SparkPlan): 
Boolean = plan match {
+case _: ArrowFileSourceScanExec => true
+case _ => false
+  }
 }
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowScanReplaceRule.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowScanReplaceRule.scala
new file mode 100644
index 0..2b7c4b1da
--- /dev/null
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowScanReplaceRule.scala
@@ -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.
+ */
+package org.apache.gluten.extension
+
+import org.apache.gluten.datasource.ArrowCSVFileFormat
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{ArrowFileSourceScanExec, 
FileSourceScanExec, SparkPlan}
+
+case class ArrowScanReplaceRule(spark: SparkSession) extends Rule[SparkPlan] {
+  override def apply(plan: SparkPlan): SparkPlan = {
+plan.transformUp {
+  case plan: FileSourceScanExec if 
plan.relation.fileFormat.isInstanceOf[ArrowCSVFileFormat] =>
+ArrowFileSourceScanExec(plan)
+  case p => p
+}
+
+  }
+}
diff --git 
a/gluten-core/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala
 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala
similarity index 

(incubator-gluten) branch main updated: [GLUTEN-5649][VL] Fix NullPointerException when collect_list / collect_set are partially fallen back (#5655)

2024-05-11 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 2efa2e657 [GLUTEN-5649][VL] Fix NullPointerException when collect_list 
/ collect_set are partially fallen back (#5655)
2efa2e657 is described below

commit 2efa2e657f2b0cdd43adc9e57963fa17bd7bf5b6
Author: Hongze Zhang 
AuthorDate: Sat May 11 16:16:48 2024 +0800

[GLUTEN-5649][VL] Fix NullPointerException when collect_list / collect_set 
are partially fallen back (#5655)

Fixes #5649. Added vanilla implementation of velox_collect_list and 
velox_collect_set.

Velox backend's collect_list / collect_set implementations require for 
ARRAY intermediate data however Spark uses BINARY. To address this we did some 
tricks to forcibly modify the physical plan to change the output schema of 
partial aggregate operator to align with Velox, but that way the actual 
information for the two functions in Velox backend is still hidden from query 
plan so advanced optimizations or compatibility checks are made difficult 
during planning phase.

This patch adds new functions velox_collect_list / velox_collect_set to 
correctly map to Velox backend's implementation for the two functions and does 
essential code cleanup and refactors.
---
 .../clickhouse/CHSparkPlanExecApi.scala|   8 +-
 .../gluten/backendsapi/velox/VeloxBackend.scala|   8 --
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  |  13 +-
 .../execution/HashAggregateExecTransformer.scala   |  29 ++---
 .../expression}/aggregate/HLLAdapter.scala |   5 +-
 .../gluten/expression/aggregate/VeloxCollect.scala |  70 +++
 .../BloomFilterMightContainJointRewriteRule.scala  |   3 +-
 .../gluten/extension/CollectRewriteRule.scala  | 106 
 .../extension}/FlushableHashAggregateRule.scala|   2 +-
 .../extension/HLLRewriteRule.scala}|  48 +++
 .../gluten/utils/VeloxIntermediateData.scala   |   3 -
 .../apache/gluten/execution/FallbackSuite.scala|  16 ++-
 .../execution/VeloxAggregateFunctionsSuite.scala   |  79 
 .../execution/VeloxWindowExpressionSuite.scala |  20 ++-
 cpp/velox/substrait/VeloxSubstraitSignature.cc |   1 +
 .../gluten/backendsapi/BackendSettingsApi.scala|   4 -
 .../gluten/expression/ExpressionMappings.scala |   2 -
 .../extension/columnar/TransformHintRule.scala |   1 +
 .../extension/columnar/enumerated/RasOffload.scala |   1 +
 .../columnar/rewrite/RewriteCollect.scala  | 140 -
 .../{ => columnar/rewrite}/RewriteIn.scala |   4 +-
 .../columnar/rewrite/RewriteSingleNode.scala   |  10 +-
 .../rewrite/RewriteTypedImperativeAggregate.scala  |  72 ---
 .../columnar/validator/FallbackInjects.scala   |  38 ++
 .../extension/columnar/validator/Validators.scala  |  14 +++
 .../apache/gluten/utils/BackendTestSettings.scala  |  82 +---
 .../gluten/utils/velox/VeloxTestSettings.scala |   7 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |   7 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |   7 +-
 .../spark/sql/GlutenDataFrameAggregateSuite.scala  |  79 ++--
 .../gluten/utils/velox/VeloxTestSettings.scala |   7 +-
 31 files changed, 522 insertions(+), 364 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 030648b06..465041621 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -40,7 +40,7 @@ import 
org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRew
 import org.apache.spark.sql.catalyst.catalog.BucketSpec
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, 
CollectList, CollectSet}
 import org.apache.spark.sql.catalyst.optimizer.BuildSide
 import org.apache.spark.sql.catalyst.plans.JoinType
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
@@ -631,7 +631,11 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
 
   /** Define backend specfic expression mappings. */
   override def extraExpressionMappings: Seq[Sig] = {
-SparkShimLoader.getSparkShims.bloomFilterExpressionMappings()
+List(
+  Sig[CollectList](ExpressionNames.COLLECT_LIST),
+  Sig[CollectSet](ExpressionNames.COLLECT_

(incubator-gluten) branch main updated: [GLUTEN-5414] [VL] Fix and enable arrow native memory pool track in CSV scan (#5683)

2024-05-11 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 3f95dbffd [GLUTEN-5414] [VL] Fix and enable arrow native memory pool 
track in CSV scan (#5683)
3f95dbffd is described below

commit 3f95dbffd79d225af596427494ff7a4690935275
Author: Jin Chengcheng 
AuthorDate: Sat May 11 14:50:39 2024 +0800

[GLUTEN-5414] [VL] Fix and enable arrow native memory pool track in CSV 
scan (#5683)
---
 ep/build-velox/src/modify_arrow.patch  | 27 ++
 .../scala/org/apache/gluten/utils/ArrowUtil.scala  |  4 ++--
 2 files changed, 29 insertions(+), 2 deletions(-)

diff --git a/ep/build-velox/src/modify_arrow.patch 
b/ep/build-velox/src/modify_arrow.patch
index 64d92725d..5814958a9 100644
--- a/ep/build-velox/src/modify_arrow.patch
+++ b/ep/build-velox/src/modify_arrow.patch
@@ -30,6 +30,33 @@ index a24f272fe..e25f78c85 100644
  #include 
  #include 
  #include 
+diff --git a/java/dataset/src/main/cpp/jni_wrapper.cc 
b/java/dataset/src/main/cpp/jni_wrapper.cc
+index d2d976677..d7dd01ecd 100644
+--- a/java/dataset/src/main/cpp/jni_wrapper.cc
 b/java/dataset/src/main/cpp/jni_wrapper.cc
+@@ -126,20 +126,14 @@ class ReserveFromJava : public 
arrow::dataset::jni::ReservationListener {
+   : vm_(vm), java_reservation_listener_(java_reservation_listener) {}
+ 
+   arrow::Status OnReservation(int64_t size) override {
+-JNIEnv* env;
+-if (vm_->GetEnv(reinterpret_cast(), JNI_VERSION) != JNI_OK) {
+-  return arrow::Status::Invalid("JNIEnv was not attached to current 
thread");
+-}
++JNIEnv* env = arrow::dataset::jni::GetEnvOrAttach(vm_);
+ env->CallObjectMethod(java_reservation_listener_, reserve_memory_method, 
size);
+ RETURN_NOT_OK(arrow::dataset::jni::CheckException(env));
+ return arrow::Status::OK();
+   }
+ 
+   arrow::Status OnRelease(int64_t size) override {
+-JNIEnv* env;
+-if (vm_->GetEnv(reinterpret_cast(), JNI_VERSION) != JNI_OK) {
+-  return arrow::Status::Invalid("JNIEnv was not attached to current 
thread");
+-}
++JNIEnv* env = arrow::dataset::jni::GetEnvOrAttach(vm_);
+ env->CallObjectMethod(java_reservation_listener_, 
unreserve_memory_method, size);
+ RETURN_NOT_OK(arrow::dataset::jni::CheckException(env));
+ return arrow::Status::OK();
 diff --git a/java/pom.xml b/java/pom.xml
 index a8328576b..57f282c6c 100644
 --- a/java/pom.xml
diff --git a/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowUtil.scala 
b/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowUtil.scala
index 4579e015b..26bebcfae 100644
--- a/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowUtil.scala
+++ b/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowUtil.scala
@@ -18,6 +18,7 @@ package org.apache.gluten.utils
 
 import org.apache.gluten.exception.SchemaMismatchException
 import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
+import org.apache.gluten.memory.arrow.pool.ArrowNativeMemoryPool
 import org.apache.gluten.vectorized.ArrowWritableColumnVector
 
 import org.apache.spark.internal.Logging
@@ -33,7 +34,6 @@ import org.apache.spark.sql.vectorized.{ColumnarBatch, 
ColumnVector}
 
 import org.apache.arrow.c.{ArrowSchema, CDataDictionaryProvider, Data}
 import org.apache.arrow.dataset.file.{FileFormat, FileSystemDatasetFactory}
-import org.apache.arrow.dataset.jni.NativeMemoryPool
 import org.apache.arrow.memory.BufferAllocator
 import org.apache.arrow.vector.ipc.message.ArrowRecordBatch
 import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
@@ -144,7 +144,7 @@ object ArrowUtil extends Logging {
 val allocator = ArrowBufferAllocators.contextInstance()
 val factory = new FileSystemDatasetFactory(
   allocator,
-  NativeMemoryPool.getDefault, // TODO: wait to change
+  ArrowNativeMemoryPool.arrowPool("FileSystemDatasetFactory"),
   format,
   rewriteUri(encodedUri))
 factory


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Substrait-to-Velox: Support nested complex type signature parsing

2024-05-09 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new a9fa77224 [VL] Substrait-to-Velox: Support nested complex type 
signature parsing
a9fa77224 is described below

commit a9fa7722405ddf72cc1aaba026aafebaf771748e
Author: Hongze Zhang 
AuthorDate: Thu May 9 14:02:36 2024 +0800

[VL] Substrait-to-Velox: Support nested complex type signature parsing
---
 cpp/velox/substrait/VeloxSubstraitSignature.cc | 103 +
 cpp/velox/tests/VeloxSubstraitSignatureTest.cc |   7 ++
 2 files changed, 78 insertions(+), 32 deletions(-)

diff --git a/cpp/velox/substrait/VeloxSubstraitSignature.cc 
b/cpp/velox/substrait/VeloxSubstraitSignature.cc
index 34e0df6de..ee7c5f513 100644
--- a/cpp/velox/substrait/VeloxSubstraitSignature.cc
+++ b/cpp/velox/substrait/VeloxSubstraitSignature.cc
@@ -72,6 +72,48 @@ std::string 
VeloxSubstraitSignature::toSubstraitSignature(const TypePtr& type) {
   }
 }
 
+namespace {
+using index = std::string::size_type;
+
+index findEnclosingPos(std::string text, index from, char left, char right) {
+  VELOX_CHECK(left != right)
+  VELOX_CHECK(text.at(from) == left)
+  int32_t stackedLeftChars = 0;
+  for (index idx = from; idx < text.size(); idx++) {
+const char ch = text.at(idx);
+if (ch == left) {
+  stackedLeftChars++;
+}
+if (ch == right) {
+  stackedLeftChars--;
+}
+if (stackedLeftChars == 0) {
+  return idx;
+}
+  }
+  VELOX_FAIL("Unable to find enclose character from text: " + text)
+}
+
+index findSansNesting(std::string text, index from, char target, char left, 
char right) {
+  VELOX_CHECK(left != right)
+  VELOX_CHECK(target != left && target != right)
+  int32_t stackedLeftChars = 0;
+  for (index idx = from; idx < text.size(); idx++) {
+const char ch = text.at(idx);
+if (ch == left) {
+  stackedLeftChars++;
+}
+if (ch == right) {
+  stackedLeftChars--;
+}
+if (ch == target && stackedLeftChars == 0) {
+  return idx;
+}
+  }
+  return std::string::npos;
+}
+} // namespace
+
 TypePtr VeloxSubstraitSignature::fromSubstraitSignature(const std::string& 
signature) {
   if (signature == "bool") {
 return BOOLEAN();
@@ -123,7 +165,7 @@ TypePtr 
VeloxSubstraitSignature::fromSubstraitSignature(const std::string& signa
 
   auto parseNestedTypeSignature = [&](const std::string& signature) -> 
std::vector {
 auto start = signature.find_first_of('<');
-auto end = signature.find_last_of('>');
+auto end = findEnclosingPos(signature, start, '<', '>');
 VELOX_CHECK(
 end - start > 1,
 "Native validation failed due to: more information is needed to create 
nested type for {}",
@@ -132,30 +174,25 @@ TypePtr 
VeloxSubstraitSignature::fromSubstraitSignature(const std::string& signa
 std::string childrenTypes = signature.substr(start + 1, end - start - 1);
 
 // Split the types with delimiter.
-std::string delimiter = ",";
-std::size_t pos;
+const char delimiter = ',';
 std::vector types;
-while ((pos = childrenTypes.find(delimiter)) != std::string::npos) {
-  auto typeStr = childrenTypes.substr(0, pos);
-  std::size_t endPos = pos;
-  if (startWith(typeStr, "dec") || startWith(typeStr, "struct") || 
startWith(typeStr, "map") ||
-  startWith(typeStr, "list")) {
-endPos = childrenTypes.find(">") + 1;
-if (endPos > pos) {
-  typeStr += childrenTypes.substr(pos, endPos - pos);
-} else {
-  // For nested case, the end '>' could missing,
-  // so the last position is treated as end.
-  typeStr += childrenTypes.substr(pos);
-  endPos = childrenTypes.size();
-}
+size_t typeStart = 0;
+while (true) {
+  if (typeStart == childrenTypes.size()) {
+break;
+  }
+  VELOX_CHECK(typeStart < childrenTypes.size())
+  const size_t typeEnd = findSansNesting(childrenTypes, typeStart, 
delimiter, '<', '>');
+  if (typeEnd == std::string::npos) {
+std::string typeStr = childrenTypes.substr(typeStart);
+types.emplace_back(fromSubstraitSignature(typeStr));
+break;
   }
+  std::string typeStr = childrenTypes.substr(typeStart, typeEnd - 
typeStart);
   types.emplace_back(fromSubstraitSignature(typeStr));
-  childrenTypes.erase(0, endPos + delimiter.length());
-}
-if (childrenTypes.size() > 0 && !startWith(childrenTypes, ">")) {
-  types.emplace_back(fromSubstraitSignature(childrenTypes));
+  typeStart = typeEnd + 1;
 }
+
 return types;
   };
 
@@ -172,6 +209,10 @@ TypePtr 
VeloxSubstrai

(incubator-gluten) branch main updated: [GLUTEN-5656][CORE] Avoid executing subqueries with complex data type during validation

2024-05-08 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new a7de1955a [GLUTEN-5656][CORE] Avoid executing subqueries with complex 
data type during validation
a7de1955a is described below

commit a7de1955a507c185fd7c0ef44a3a36b713115582
Author: Hongze Zhang 
AuthorDate: Thu May 9 12:50:57 2024 +0800

[GLUTEN-5656][CORE] Avoid executing subqueries with complex data type 
during validation

Fixes #5656
---
 .../gluten/expression/ScalarSubqueryTransformer.scala  | 18 +-
 1 file changed, 1 insertion(+), 17 deletions(-)

diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/expression/ScalarSubqueryTransformer.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/expression/ScalarSubqueryTransformer.scala
index 61b1e4382..534bde3b3 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/expression/ScalarSubqueryTransformer.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/expression/ScalarSubqueryTransformer.scala
@@ -20,17 +20,13 @@ import 
org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode
 
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution.{BaseSubqueryExec, ScalarSubquery}
-import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType}
 
 case class ScalarSubqueryTransformer(plan: BaseSubqueryExec, exprId: ExprId, 
query: ScalarSubquery)
   extends ExpressionTransformer {
 
   override def doTransform(args: java.lang.Object): ExpressionNode = {
 // don't trigger collect when in validation phase
-if (
-  TransformerState.underValidationState &&
-  !valueSensitiveDataType(query.dataType)
-) {
+if (TransformerState.underValidationState) {
   return ExpressionBuilder.makeLiteral(null, query.dataType, true)
 }
 // the first column in first row from `query`.
@@ -50,16 +46,4 @@ case class ScalarSubqueryTransformer(plan: BaseSubqueryExec, 
exprId: ExprId, que
 }
 ExpressionBuilder.makeLiteral(result, query.dataType, result == null)
   }
-
-  /**
-   * DataTypes which supported or not depend on actual value
-   *
-   * @param dataType
-   * @return
-   */
-  def valueSensitiveDataType(dataType: DataType): Boolean = {
-dataType.isInstanceOf[MapType] ||
-dataType.isInstanceOf[ArrayType] ||
-dataType.isInstanceOf[StructType]
-  }
 }


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [GLUTEN-5414] [VL] Support Arrow native memory pool usage track (#5550)

2024-05-07 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new cee1f3bd8 [GLUTEN-5414] [VL] Support Arrow native memory pool usage 
track (#5550)
cee1f3bd8 is described below

commit cee1f3bd869340c156feb0cdaf0a867f32343d54
Author: Jin Chengcheng 
AuthorDate: Wed May 8 11:35:25 2024 +0800

[GLUTEN-5414] [VL] Support Arrow native memory pool usage track (#5550)
---
 .../gluten/execution/RowToVeloxColumnarExec.scala  |  2 +-
 .../python/ColumnarArrowEvalPythonExec.scala   |  2 +-
 .../org/apache/gluten/utils/DatasourceUtil.scala   |  2 +-
 .../execution/ColumnarCachedBatchSerializer.scala  |  2 +-
 .../execution/VeloxColumnarWriteFilesExec.scala|  2 +-
 .../velox/VeloxFormatWriterInjects.scala   |  2 +-
 .../VeloxCelebornColumnarBatchSerializer.scala |  2 +-
 gluten-data/pom.xml| 28 
 .../gluten/columnarbatch/ColumnarBatches.java  |  2 +-
 .../alloc}/ArrowBufferAllocators.java  |  2 +-
 .../alloc}/ManagedAllocationListener.java  |  2 +-
 .../memory/arrow/pool/ArrowNativeMemoryPool.java   | 75 ++
 .../arrow/pool/ArrowReservationListener.java}  | 33 +-
 .../vectorized/ArrowWritableColumnVector.java  |  2 +-
 .../gluten/vectorized/ColumnarBatchInIterator.java |  2 +-
 .../org/apache/gluten/utils/ImplicitClass.scala|  2 +-
 .../vectorized/ColumnarBatchSerializer.scala   |  2 +-
 .../sql/execution/ColumnarBuildSideRelation.scala  |  2 +-
 .../spark/sql/execution/utils/ExecUtil.scala   |  2 +-
 .../apache/spark/sql/utils/SparkVectorUtil.scala   |  2 +-
 20 files changed, 138 insertions(+), 32 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala
index 22ca020d1..be1bc64e2 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala
@@ -20,7 +20,7 @@ import org.apache.gluten.backendsapi.BackendsApiManager
 import org.apache.gluten.columnarbatch.ColumnarBatches
 import org.apache.gluten.exception.GlutenException
 import org.apache.gluten.exec.Runtimes
-import org.apache.gluten.memory.arrowalloc.ArrowBufferAllocators
+import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
 import org.apache.gluten.memory.nmm.NativeMemoryManagers
 import org.apache.gluten.utils.{ArrowAbiUtil, Iterators}
 import org.apache.gluten.vectorized._
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
index f2beef6ca..77ef1c642 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala
@@ -18,7 +18,7 @@ package org.apache.spark.api.python
 
 import org.apache.gluten.columnarbatch.ColumnarBatches
 import org.apache.gluten.extension.GlutenPlan
-import org.apache.gluten.memory.arrowalloc.ArrowBufferAllocators
+import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
 import org.apache.gluten.utils.Iterators
 import org.apache.gluten.vectorized.ArrowWritableColumnVector
 
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/utils/DatasourceUtil.scala 
b/backends-velox/src/main/scala/org/apache/gluten/utils/DatasourceUtil.scala
index f61cdb8e9..6150507b4 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/DatasourceUtil.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/DatasourceUtil.scala
@@ -17,7 +17,7 @@
 package org.apache.gluten.utils
 
 import org.apache.gluten.datasource.DatasourceJniWrapper
-import org.apache.gluten.memory.arrowalloc.ArrowBufferAllocators
+import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
 import org.apache.gluten.memory.nmm.NativeMemoryManagers
 
 import org.apache.spark.sql.types.StructType
diff --git 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala
 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala
index 88678cb5e..7385c53d6 100644
--- 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala
+++ 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala
@@ -21,7 +21,7 @@ import org.apache.gluten.backendsapi.BackendsApiManager
 import org.apache.gluten.columnarbatch.ColumnarBatches
 import org.apache.gluten.exec.Runtimes
 import

(incubator-gluten) branch main updated: [VL] Add a bad test case that final aggregate of collect_list is fallen back while partial aggregate is not (#5649)

2024-05-07 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new e633887c8 [VL] Add a bad test case that final aggregate of 
collect_list is fallen back while partial aggregate is not (#5649)
e633887c8 is described below

commit e633887c8447cb136015da5a817898a759739697
Author: Hongze Zhang 
AuthorDate: Wed May 8 11:23:58 2024 +0800

[VL] Add a bad test case that final aggregate of collect_list is fallen 
back while partial aggregate is not (#5649)
---
 .../apache/gluten/execution/FallbackSuite.scala| 34 ++
 1 file changed, 34 insertions(+)

diff --git 
a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala 
b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala
index 91024da1c..fbad525a2 100644
--- 
a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala
+++ 
b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala
@@ -106,6 +106,40 @@ class FallbackSuite extends 
VeloxWholeStageTransformerSuite with AdaptiveSparkPl
 }
   }
 
+  // java.lang.NullPointerException
+  ignore("fallback final aggregate of collect_list") {
+withSQLConf(
+  GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1",
+  GlutenConfig.COLUMNAR_FALLBACK_IGNORE_ROW_TO_COLUMNAR.key -> "false",
+  GlutenConfig.EXPRESSION_BLACK_LIST.key -> "element_at"
+) {
+  runQueryAndCompare(
+"SELECT sum(ele) FROM (SELECT c1, element_at(collect_list(c2), 1) as 
ele FROM tmp1 " +
+  "GROUP BY c1)") {
+df =>
+  val columnarToRow = 
collectColumnarToRow(df.queryExecution.executedPlan)
+  assert(columnarToRow == 1)
+  }
+}
+  }
+
+  // java.lang.NullPointerException
+  ignore("fallback final aggregate of collect_set") {
+withSQLConf(
+  GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1",
+  GlutenConfig.COLUMNAR_FALLBACK_IGNORE_ROW_TO_COLUMNAR.key -> "false",
+  GlutenConfig.EXPRESSION_BLACK_LIST.key -> "element_at"
+) {
+  runQueryAndCompare(
+"SELECT sum(ele) FROM (SELECT c1, element_at(collect_set(c2), 1) as 
ele FROM tmp1 " +
+  "GROUP BY c1)") {
+df =>
+  val columnarToRow = 
collectColumnarToRow(df.queryExecution.executedPlan)
+  assert(columnarToRow == 1)
+  }
+}
+  }
+
   test("fallback with AQE read") {
 runQueryAndCompare(
   """


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] RAS: Include rewrite rules used by RewriteSparkPlanRulesManager in EnumeratedTransform (#5575)

2024-05-07 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 18af4bc3c [VL] RAS: Include rewrite rules used by 
RewriteSparkPlanRulesManager in EnumeratedTransform (#5575)
18af4bc3c is described below

commit 18af4bc3ce4c3e685ad63c869880f8b63d48dc1c
Author: Hongze Zhang 
AuthorDate: Tue May 7 14:49:03 2024 +0800

[VL] RAS: Include rewrite rules used by RewriteSparkPlanRulesManager in 
EnumeratedTransform (#5575)
---
 .../clickhouse/CHSparkPlanExecApi.scala|  3 +-
 .../execution/CHHashAggregateExecTransformer.scala |  6 +-
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  |  3 +-
 .../execution/HashAggregateExecTransformer.scala   | 14 ++--
 .../gluten/backendsapi/SparkPlanExecApi.scala  |  1 -
 .../HashAggregateExecBaseTransformer.scala | 30 +--
 .../org/apache/gluten/extension/RewriteIn.scala|  7 +-
 .../extension/columnar/MiscColumnarRules.scala | 18 ++--
 ...ormSingleNode.scala => OffloadSingleNode.scala} | 95 --
 .../extension/columnar/TransformHintRule.scala | 33 +---
 .../columnar/enumerated/ConditionedRule.scala  | 51 
 .../columnar/enumerated/EnumeratedApplier.scala|  5 +-
 .../columnar/enumerated/EnumeratedTransform.scala  | 56 -
 .../columnar/enumerated/PushFilterToScan.scala | 27 +++---
 .../extension/columnar/enumerated/RasOffload.scala | 84 +++
 ...ntAggregate.scala => RasOffloadAggregate.scala} | 35 ++--
 ...mplementFilter.scala => RasOffloadFilter.scala} | 16 ++--
 .../{FilterRemoveRule.scala => RemoveFilter.scala} |  2 +-
 .../columnar/heuristic/HeuristicApplier.scala  |  1 +
 .../{ => rewrite}/PullOutPostProject.scala | 14 ++--
 .../columnar/{ => rewrite}/PullOutPreProject.scala |  9 +-
 .../{ => columnar/rewrite}/RewriteCollect.scala|  9 +-
 .../{ => rewrite}/RewriteMultiChildrenCount.scala  |  7 +-
 .../columnar/rewrite/RewriteSingleNode.scala   | 48 +++
 .../RewriteSparkPlanRulesManager.scala | 17 ++--
 .../RewriteTypedImperativeAggregate.scala  |  7 +-
 .../gluten/planner/cost/GlutenCostModel.scala  |  6 +-
 .../GlutenFormatWriterInjectsBase.scala|  3 +-
 28 files changed, 295 insertions(+), 312 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 64090af28..a9a12a3ea 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -204,10 +204,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
 
   /** Generate HashAggregateExecPullOutHelper */
   override def genHashAggregateExecPullOutHelper(
-  groupingExpressions: Seq[NamedExpression],
   aggregateExpressions: Seq[AggregateExpression],
   aggregateAttributes: Seq[Attribute]): HashAggregateExecPullOutBaseHelper 
=
-CHHashAggregateExecPullOutHelper(groupingExpressions, 
aggregateExpressions, aggregateAttributes)
+CHHashAggregateExecPullOutHelper(aggregateExpressions, aggregateAttributes)
 
   /**
* If there are expressions (not field reference) in the partitioning's 
children, add a projection
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala
index 82c492f4c..d4f2f9eb3 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala
@@ -411,13 +411,9 @@ case class CHHashAggregateExecTransformer(
 }
 
 case class CHHashAggregateExecPullOutHelper(
-groupingExpressions: Seq[NamedExpression],
 aggregateExpressions: Seq[AggregateExpression],
 aggregateAttributes: Seq[Attribute])
-  extends HashAggregateExecPullOutBaseHelper(
-groupingExpressions,
-aggregateExpressions,
-aggregateAttributes) {
+  extends HashAggregateExecPullOutBaseHelper {
 
   /** This method calculates the output attributes of Aggregation. */
   override protected def getAttrForAggregateExprs: List[Attribute] = {
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
index a55aa1817..0a9f3ef65 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
+++ 
b

(incubator-gluten) branch main updated: [VL] CI: Gluten-it: Print planning time as well as execution time in test report (#5616)

2024-05-06 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 717b263d0 [VL] CI: Gluten-it: Print planning time as well as execution 
time in test report (#5616)
717b263d0 is described below

commit 717b263d01310d94c0fa4ec506d148ef00367448
Author: Hongze Zhang 
AuthorDate: Tue May 7 09:25:38 2024 +0800

[VL] CI: Gluten-it: Print planning time as well as execution time in test 
report (#5616)
---
 .../integration/tpc/action/Parameterized.scala | 55 +---
 .../gluten/integration/tpc/action/Queries.scala| 34 +++---
 .../integration/tpc/action/QueriesCompare.scala| 61 --
 .../integration/tpc/action/TableFormatter.scala| 74 ++
 .../scala/org/apache/spark/sql/QueryRunner.scala   | 12 +++-
 5 files changed, 183 insertions(+), 53 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Parameterized.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Parameterized.scala
index b4f7a5394..6fc4e66d6 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Parameterized.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/Parameterized.scala
@@ -25,7 +25,7 @@ import 
org.apache.gluten.integration.tpc.action.Actions.QuerySelector
 
 import scala.collection.immutable.Map
 import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 
 class Parameterized(
 scale: Double,
@@ -189,30 +189,15 @@ case class TestResultLine(
 succeed: Boolean,
 coordinate: Coordinate,
 rowCount: Option[Long],
+planningTimeMillis: Option[Long],
 executionTimeMillis: Option[Long],
 metrics: Map[String, Long],
 errorMessage: Option[String])
 
-case class TestResultLines(
-dimNames: Seq[String],
-metricNames: Seq[String],
-lines: Iterable[TestResultLine]) {
-  def print(): Unit = {
-var fmt = "|%15s|%15s"
-for (_ <- dimNames.indices) {
-  fmt = fmt + "|%20s"
-}
-for (_ <- metricNames.indices) {
-  fmt = fmt + "|%35s"
-}
-fmt = fmt + "|%30s|%30s|\n"
-val fields = ArrayBuffer[String]("Query ID", "Succeed")
-dimNames.foreach(dimName => fields.append(dimName))
-metricNames.foreach(metricName => fields.append(metricName))
-fields.append("Row Count")
-fields.append("Query Time (Millis)")
-printf(fmt, fields: _*)
-lines.foreach { line =>
+object TestResultLine {
+  class Parser(dimNames: Seq[String], metricNames: Seq[String])
+  extends TableFormatter.RowParser[TestResultLine] {
+override def parse(line: TestResultLine): Seq[Any] = {
   val values = ArrayBuffer[Any](line.queryId, line.succeed)
   dimNames.foreach { dimName =>
 val coordinate = line.coordinate.coordinate
@@ -226,9 +211,32 @@ case class TestResultLines(
 values.append(metrics.getOrElse(metricName, "N/A"))
   }
   values.append(line.rowCount.getOrElse("N/A"))
+  values.append(line.planningTimeMillis.getOrElse("N/A"))
   values.append(line.executionTimeMillis.getOrElse("N/A"))
-  printf(fmt, values: _*)
+  values
+}
+  }
+}
+
+case class TestResultLines(
+dimNames: Seq[String],
+metricNames: Seq[String],
+lines: Iterable[TestResultLine]) {
+  def print(): Unit = {
+val fields = ListBuffer[String]("Query ID", "Succeed")
+dimNames.foreach(dimName => fields.append(dimName))
+metricNames.foreach(metricName => fields.append(metricName))
+fields.append("Row Count")
+fields.append("Planning Time (Millis)")
+fields.append("Query Time (Millis)")
+val formatter = TableFormatter.create[TestResultLine](fields: _*)(
+  new TestResultLine.Parser(dimNames, metricNames))
+
+lines.foreach { line =>
+  formatter.appendRow(line)
 }
+
+formatter.print(System.out)
   }
 }
 
@@ -257,6 +265,7 @@ object Parameterized {
 succeed = true,
 coordinate,
 Some(resultRows.length),
+Some(result.planningTimeMillis),
 Some(result.executionTimeMillis),
 result.metrics,
 None)
@@ -266,7 +275,7 @@ object Parameterized {
 println(
   s"Error running query $id. " +
 s" Error: ${error.get}")
-TestResultLine(id, succeed = false, coordinate, None, None, Map.empty, 
error)
+TestResultLine(id, succeed = false, coordinate, None, None, None, 
Map.empty, error)
 }
   }
 
diff --git 
a/tools/gluten-it/com

(incubator-gluten) branch main updated: [VL] CI: Reformat gluten-it code with Spark331's scalafmt configuration (#5615)

2024-05-06 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 7020ed376 [VL] CI: Reformat gluten-it code with Spark331's scalafmt 
configuration (#5615)
7020ed376 is described below

commit 7020ed3768ae1315481d6091a6aec33e3f93b66f
Author: Hongze Zhang 
AuthorDate: Mon May 6 17:06:45 2024 +0800

[VL] CI: Reformat gluten-it code with Spark331's scalafmt configuration 
(#5615)
---
 .../apache/gluten/integration/tpc/Constants.scala  |  28 ++--
 .../apache/gluten/integration/tpc/DataGen.scala|  43 ---
 .../apache/gluten/integration/tpc/ShimUtils.scala  |  12 +-
 .../apache/gluten/integration/tpc/TpcRunner.scala  |  25 ++--
 .../apache/gluten/integration/tpc/TpcSuite.scala   |  15 ++-
 .../integration/tpc/action/Parameterized.scala | 142 +
 .../gluten/integration/tpc/action/Queries.scala|  72 +--
 .../integration/tpc/action/QueriesCompare.scala| 105 +++
 .../gluten/integration/tpc/ds/TpcdsDataGen.scala   | 112 +++-
 .../gluten/integration/tpc/ds/TpcdsSuite.scala |  44 ---
 .../gluten/integration/tpc/h/TpchDataGen.scala | 141 
 .../gluten/integration/tpc/h/TpchSuite.scala   |  35 +++--
 .../history/GlutenItHistoryServerPlugin.scala  |  85 ++--
 .../spark/deploy/history/HistoryServerHelper.scala |  16 +--
 .../scala/org/apache/spark/sql/ConfUtils.scala |  10 +-
 .../scala/org/apache/spark/sql/QueryRunner.scala   |  25 ++--
 .../scala/org/apache/spark/sql/TestUtils.scala |  29 ++---
 17 files changed, 430 insertions(+), 509 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/Constants.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/Constants.scala
index 7564f6dce..d39a16c32 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/Constants.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/Constants.scala
@@ -18,7 +18,14 @@ package org.apache.gluten.integration.tpc
 
 import org.apache.spark.SparkConf
 import org.apache.spark.sql.TypeUtils
-import org.apache.spark.sql.types.{DateType, DecimalType, DoubleType, 
IntegerType, LongType, StringType}
+import org.apache.spark.sql.types.{
+  DateType,
+  DecimalType,
+  DoubleType,
+  IntegerType,
+  LongType,
+  StringType
+}
 
 import java.sql.Date
 
@@ -33,16 +40,15 @@ object Constants {
 .set("spark.shuffle.manager", 
"org.apache.spark.shuffle.sort.ColumnarShuffleManager")
 .set("spark.sql.optimizer.runtime.bloomFilter.enabled", "true")
 
.set("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold",
 "0")
-.set(
-  "spark.gluten.sql.columnar.physicalJoinOptimizeEnable",
-  "false"
-) // q72 slow if false, q64 fails if true
+.set("spark.gluten.sql.columnar.physicalJoinOptimizeEnable", "false") // 
q72 slow if false, q64 fails if true
 
   val VELOX_WITH_CELEBORN_CONF: SparkConf = new SparkConf(false)
 .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "true")
 .set("spark.sql.parquet.enableVectorizedReader", "true")
 .set("spark.plugins", "org.apache.gluten.GlutenPlugin")
-.set("spark.shuffle.manager", 
"org.apache.spark.shuffle.gluten.celeborn.CelebornShuffleManager")
+.set(
+  "spark.shuffle.manager",
+  "org.apache.spark.shuffle.gluten.celeborn.CelebornShuffleManager")
 .set("spark.celeborn.shuffle.writer", "hash")
 .set("spark.celeborn.push.replicate.enabled", "false")
 .set("spark.celeborn.client.shuffle.compression.codec", "none")
@@ -51,10 +57,7 @@ object Constants {
 .set("spark.dynamicAllocation.enabled", "false")
 .set("spark.sql.optimizer.runtime.bloomFilter.enabled", "true")
 
.set("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold",
 "0")
-.set(
-  "spark.gluten.sql.columnar.physicalJoinOptimizeEnable",
-  "false"
-) // q72 slow if false, q64 fails if true
+.set("spark.gluten.sql.columnar.physicalJoinOptimizeEnable", "false") // 
q72 slow if false, q64 fails if true
 .set("spark.celeborn.push.data.timeout", "600s")
 .set("spark.celeborn.push.limit.inFlight.timeout", "1200s")
 
@@ -72,10 +75,7 @@ object Constants {
 .set("spark.dynamicAllocation.enabled", "false")
 .set("spark.sql.optimizer.runtime.bloomFi

(incubator-gluten) branch main updated: [VL] Bloom-filter expressions are unexpectedly fallen back (#5579)

2024-04-29 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 6a04248dc [VL] Bloom-filter expressions are unexpectedly fallen back 
(#5579)
6a04248dc is described below

commit 6a04248dc3d7f6660a6a4b0248908516c6aef53a
Author: Hongze Zhang 
AuthorDate: Tue Apr 30 08:12:05 2024 +0800

[VL] Bloom-filter expressions are unexpectedly fallen back (#5579)
---
 .../backendsapi/velox/VeloxSparkPlanExecApi.scala  | 14 ++---
 .../BloomFilterMightContainJointRewriteRule.scala  | 24 +-
 2 files changed, 20 insertions(+), 18 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
index 8318ac2d5..a55aa1817 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala
@@ -66,7 +66,6 @@ import javax.ws.rs.core.UriBuilder
 import java.lang.{Long => JLong}
 import java.util.{Map => JMap}
 
-import scala.collection.mutable
 import scala.collection.mutable.ListBuffer
 
 class VeloxSparkPlanExecApi extends SparkPlanExecApi {
@@ -734,19 +733,18 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
*
* @return
*/
-  override def genExtendedOptimizers(): List[SparkSession => 
Rule[LogicalPlan]] = {
-val buf = mutable.ListBuffer[SparkSession => Rule[LogicalPlan]]()
-buf += AggregateFunctionRewriteRule.apply
-buf += BloomFilterMightContainJointRewriteRule.apply
-buf.toList
-  }
+  override def genExtendedOptimizers(): List[SparkSession => 
Rule[LogicalPlan]] = List(
+AggregateFunctionRewriteRule.apply
+  )
 
   /**
* Generate extended columnar pre-rules, in the validation phase.
*
* @return
*/
-  override def genExtendedColumnarValidationRules(): List[SparkSession => 
Rule[SparkPlan]] = List()
+  override def genExtendedColumnarValidationRules(): List[SparkSession => 
Rule[SparkPlan]] = List(
+BloomFilterMightContainJointRewriteRule.apply
+  )
 
   /**
* Generate extended columnar pre-rules.
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/extension/BloomFilterMightContainJointRewriteRule.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/extension/BloomFilterMightContainJointRewriteRule.scala
index c8cb4cca3..deba381db 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/extension/BloomFilterMightContainJointRewriteRule.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/extension/BloomFilterMightContainJointRewriteRule.scala
@@ -22,24 +22,28 @@ import 
org.apache.gluten.expression.aggregate.VeloxBloomFilterAggregate
 import org.apache.gluten.sql.shims.SparkShimLoader
 
 import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.SparkPlan
 
-case class BloomFilterMightContainJointRewriteRule(spark: SparkSession) 
extends Rule[LogicalPlan] {
-  override def apply(plan: LogicalPlan): LogicalPlan = {
+case class BloomFilterMightContainJointRewriteRule(spark: SparkSession) 
extends Rule[SparkPlan] {
+  override def apply(plan: SparkPlan): SparkPlan = {
 if (!(GlutenConfig.getConf.enableNativeBloomFilter)) {
   return plan
 }
 val out = plan.transformWithSubqueries {
   case p =>
-p.transformExpressions {
-  case e =>
-SparkShimLoader.getSparkShims.replaceMightContain(
-  SparkShimLoader.getSparkShims
-.replaceBloomFilterAggregate(e, 
VeloxBloomFilterAggregate.apply),
-  VeloxBloomFilterMightContain.apply)
-}
+applyForNode(p)
 }
 out
   }
+
+  private def applyForNode(p: SparkPlan) = {
+p.transformExpressions {
+  case e =>
+SparkShimLoader.getSparkShims.replaceMightContain(
+  SparkShimLoader.getSparkShims
+.replaceBloomFilterAggregate(e, VeloxBloomFilterAggregate.apply),
+  VeloxBloomFilterMightContain.apply)
+}
+  }
 }


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [GLUTEN-5525][CORE] Build: `mvn clean` fails without a spark profile (#5519)

2024-04-27 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 452ebde7a [GLUTEN-5525][CORE] Build: `mvn clean` fails without a spark 
profile (#5519)
452ebde7a is described below

commit 452ebde7af84c1ba3951776349ddefdae7f8604f
Author: zhouyifan279 <88070094+zhouyifan...@users.noreply.github.com>
AuthorDate: Sun Apr 28 12:39:26 2024 +0800

[GLUTEN-5525][CORE] Build: `mvn clean` fails without a spark profile (#5519)

Closes #5525
---
 pom.xml   | 16 +++-
 shims/pom.xml | 28 +---
 2 files changed, 12 insertions(+), 32 deletions(-)

diff --git a/pom.xml b/pom.xml
index 7b1ffb8a9..f666de874 100644
--- a/pom.xml
+++ b/pom.xml
@@ -47,9 +47,14 @@
 2.12
 2.12.15
 3
-3.2
+3.4
 3.4.2
-
spark-sql-columnar-shims-spark32
+spark34
+
spark-sql-columnar-shims-spark34
+1.5.0
+delta-core
+2.4.0
+24
 0.3.2-incubating
 0.8.0
 15.0.0
@@ -129,11 +134,9 @@
 
 
   spark-3.2
-  
-true
-  
   
 3.2
+spark32
 
spark-sql-columnar-shims-spark32
 3.2.2
1.3.1
@@ -146,6 +149,7 @@
   spark-3.3
   
 3.3
+spark33
 
spark-sql-columnar-shims-spark33
 3.3.1
 
@@ -159,6 +163,7 @@
   spark-3.4
   
 3.4
+spark34
 
spark-sql-columnar-shims-spark34
 3.4.2
1.5.0
@@ -171,6 +176,7 @@
   spark-3.5
   
 3.5
+spark35
 
spark-sql-columnar-shims-spark35
 3.5.1
1.5.0
diff --git a/shims/pom.xml b/shims/pom.xml
index bf155ce48..5c17c3ec3 100644
--- a/shims/pom.xml
+++ b/shims/pom.xml
@@ -30,6 +30,7 @@
 
   
 common
+${sparkshim.module.name}
   
 
   
@@ -68,31 +69,4 @@
   
 
   
-
-  
-
-  spark-3.2
-  
-spark32
-  
-
-
-  spark-3.3
-  
-spark33
-  
-
-
-  spark-3.4
-  
-spark34
-  
-
- 
-  spark-3.5
-  
-spark35
-  
-
-  
 


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] RAS: Remove AddTransformHintRule route from EnumeratedApplier (#5552)

2024-04-27 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new f9779db1e [VL] RAS: Remove AddTransformHintRule route from 
EnumeratedApplier (#5552)
f9779db1e is described below

commit f9779db1e97873b5befe6a42c6237aa0e352731f
Author: Hongze Zhang 
AuthorDate: Sun Apr 28 08:07:35 2024 +0800

[VL] RAS: Remove AddTransformHintRule route from EnumeratedApplier (#5552)
---
 .../extension/columnar/TransformSingleNode.scala   | 33 -
 .../columnar/enumerated/ConditionedRule.scala  | 23 ++--
 .../columnar/enumerated/EnumeratedApplier.scala|  3 +-
 .../columnar/enumerated/EnumeratedTransform.scala  | 42 --
 .../columnar/enumerated/ImplementAggregate.scala   | 11 +++---
 .../columnar/enumerated/ImplementFilter.scala  | 12 ---
 .../gluten/planner/cost/GlutenCostModel.scala  | 29 +--
 7 files changed, 83 insertions(+), 70 deletions(-)

diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformSingleNode.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformSingleNode.scala
index b8f99330e..760929bbd 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformSingleNode.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformSingleNode.scala
@@ -132,22 +132,7 @@ case class TransformExchange() extends TransformSingleNode 
with LogLevelUtil {
 
 // Join transformation.
 case class TransformJoin() extends TransformSingleNode with LogLevelUtil {
-
-  /**
-   * Get the build side supported by the execution of vanilla Spark.
-   *
-   * @param plan
-   *   : shuffled hash join plan
-   * @return
-   *   the supported build side
-   */
-  private def getSparkSupportedBuildSide(plan: ShuffledHashJoinExec): 
BuildSide = {
-plan.joinType match {
-  case LeftOuter | LeftSemi => BuildRight
-  case RightOuter => BuildLeft
-  case _ => plan.buildSide
-}
-  }
+  import TransformJoin._
 
   override def impl(plan: SparkPlan): SparkPlan = {
 if (TransformHints.isNotTransformable(plan)) {
@@ -155,6 +140,7 @@ case class TransformJoin() extends TransformSingleNode with 
LogLevelUtil {
   plan match {
 case shj: ShuffledHashJoinExec =>
   if (BackendsApiManager.getSettings.recreateJoinExecOnFallback()) {
+// Since https://github.com/apache/incubator-gluten/pull/408
 // Because we manually removed the build side limitation for 
LeftOuter, LeftSemi and
 // RightOuter, need to change the build side back if this join 
fallback into vanilla
 // Spark for execution.
@@ -237,6 +223,20 @@ case class TransformJoin() extends TransformSingleNode 
with LogLevelUtil {
 
 }
 
+object TransformJoin {
+  private def getSparkSupportedBuildSide(plan: ShuffledHashJoinExec): 
BuildSide = {
+plan.joinType match {
+  case LeftOuter | LeftSemi => BuildRight
+  case RightOuter => BuildLeft
+  case _ => plan.buildSide
+}
+  }
+
+  def isLegal(plan: ShuffledHashJoinExec): Boolean = {
+plan.buildSide == getSparkSupportedBuildSide(plan)
+  }
+}
+
 // Filter transformation.
 case class TransformFilter() extends TransformSingleNode with LogLevelUtil {
   import TransformOthers._
@@ -465,6 +465,7 @@ object TransformOthers {
   }
 }
 
+// Since https://github.com/apache/incubator-gluten/pull/2701
 private def applyScanNotTransformable(plan: SparkPlan): SparkPlan = plan 
match {
   case plan: FileSourceScanExec =>
 val newPartitionFilters =
diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/ConditionedRule.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/ConditionedRule.scala
index 092d67efc..33d99f5f7 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/ConditionedRule.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/ConditionedRule.scala
@@ -37,31 +37,12 @@ object ConditionedRule {
 }
   }
 
-  trait PostCondition {
-def apply(node: SparkPlan): Boolean
-  }
-
-  object PostCondition {
-implicit class FromValidator(validator: Validator) extends PostCondition {
-  override def apply(node: SparkPlan): Boolean = {
-validator.validate(node) match {
-  case Validator.Passed => true
-  case Validator.Failed(reason) => false
-}
-  }
-}
-  }
-
-  def wrap(
-  rule: RasRule[SparkPlan],
-  pre: ConditionedRule.PreCondition,
-  post: ConditionedRule.PostCondition): RasRule[SparkPlan] = {
+  def wrap(rule: RasRule[SparkPlan], cond: ConditionedRule.PreCondition): 
RasRule[SparkPlan] = {
 new RasRule[SparkPlan] {

(incubator-gluten) branch main updated: [VL] CI: Split SF30 job to 4 jobs to speed up execution (#5526)

2024-04-25 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 46b87e5b7 [VL] CI: Split SF30 job to 4 jobs to speed up execution 
(#5526)
46b87e5b7 is described below

commit 46b87e5b7f4d49581bb38b84b7fe7f260c4121e4
Author: Hongze Zhang 
AuthorDate: Thu Apr 25 15:02:55 2024 +0800

[VL] CI: Split SF30 job to 4 jobs to speed up execution (#5526)
---
 .github/workflows/velox_docker.yml |   9 +-
 .../integration/tpc/command/Parameterized.java |  14 +--
 .../gluten/integration/tpc/command/Queries.java|  16 +--
 .../integration/tpc/command/QueriesCompare.java|  16 +--
 .../integration/tpc/command/QueriesMixin.java  | 137 +
 .../apache/gluten/integration/tpc/TpcSuite.scala   |  33 +
 .../gluten/integration/tpc/action/Actions.scala|   3 +
 .../integration/tpc/action/Parameterized.scala |  13 +-
 .../gluten/integration/tpc/action/Queries.scala|   7 +-
 .../integration/tpc/action/QueriesCompare.scala|   8 +-
 10 files changed, 172 insertions(+), 84 deletions(-)

diff --git a/.github/workflows/velox_docker.yml 
b/.github/workflows/velox_docker.yml
index 42e102ba7..20b6a5703 100644
--- a/.github/workflows/velox_docker.yml
+++ b/.github/workflows/velox_docker.yml
@@ -345,6 +345,7 @@ jobs:
   fail-fast: false
   matrix:
 spark: [ "spark-3.4" ]
+shard: [ "1/4", "2/4", "3/4", "4/4" ]
 runs-on: ubuntu-20.04
 steps:
   - name: Maximize build disk space
@@ -380,15 +381,15 @@ jobs:
   mvn -ntp clean install -P${{ matrix.spark }}
   GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh data-gen-only --local 
--benchmark-type=h -s=30.0 --threads=12
   GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh data-gen-only --local 
--benchmark-type=ds -s=30.0 --threads=12
-  - name: TPC-H / TPC-DS SF30.0 Parquet local spark3.4
+  - name: TPC-H / TPC-DS SF30.0 Parquet local ${{ matrix.spark }}
 run: |
   cd tools/gluten-it \
   && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries-compare \
 --local --preset=velox --benchmark-type=h --error-on-memleak 
-s=30.0  --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 
\
---skip-data-gen \
+--skip-data-gen --shard=${{ matrix.shard }} \
   && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries-compare \
 --local --preset=velox --benchmark-type=ds --error-on-memleak 
-s=30.0  --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 
\
---skip-data-gen
+--skip-data-gen --shard=${{ matrix.shard }}
 
   run-tpc-test-centos8-uniffle:
 needs: build-native-lib
@@ -853,4 +854,4 @@ jobs:
   cd $GITHUB_WORKSPACE/
   export MAVEN_HOME=/usr/lib/maven
   export PATH=${PATH}:${MAVEN_HOME}/bin
-  mvn -ntp clean install -Pspark-3.5 -Pbackends-velox -Pceleborn 
-Piceberg -Pdelta -Pspark-ut 
-DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" 
-DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
\ No newline at end of file
+  mvn -ntp clean install -Pspark-3.5 -Pbackends-velox -Pceleborn 
-Piceberg -Pdelta -Pspark-ut 
-DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" 
-DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
diff --git 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
index 1f94cb256..bf7d89fe6 100644
--- 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
+++ 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
@@ -43,14 +43,8 @@ public class Parameterized implements Callable {
   @CommandLine.Mixin
   private DataGenMixin dataGenMixin;
 
-  @CommandLine.Option(names = {"--queries"}, description = "Set a 
comma-separated list of query IDs to run, run all queries if not specified. 
Example: --queries=q1,q6", split = ",")
-  private String[] queries = new String[0];
-
-  @CommandLine.Option(names = {"--excluded-queries"}, description = "Set a 
comma-separated list of query IDs to exclude. Example: 
--exclude-queries=q1,q6", split = ",")
-  private String[] excludedQueries = new String[0];
-
-  @CommandLine.Option(names = {"--iterations"}, description = "How many 
iterations to run", defaultValue = "1")
-  private int iterations;
+  @CommandLine.Mixin
+  private QueriesMixin queriesMixin;
 
   @CommandLine.Option(names = {"--warmu

(incubator-gluten) branch main updated: [CORE] Untangle AddTransformHintRule to extract pre-validation code out (#5514)

2024-04-24 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new ed1f7b613 [CORE] Untangle AddTransformHintRule to extract 
pre-validation code out (#5514)
ed1f7b613 is described below

commit ed1f7b6130515b51dcefa3e47a7f461744e6dd5f
Author: Hongze Zhang 
AuthorDate: Thu Apr 25 11:48:23 2024 +0800

[CORE] Untangle AddTransformHintRule to extract pre-validation code out 
(#5514)
---
 .../extension/columnar/TransformHintRule.scala | 576 -
 .../columnar/enumerated/ConditionedRule.scala  |  70 +++
 .../columnar/enumerated/EnumeratedTransform.scala  |   8 +
 .../extension/columnar/validator/Validator.scala   |  42 ++
 .../extension/columnar/validator/Validators.scala  | 211 
 .../datasources/GlutenWriterColumnarRules.scala|   2 +
 .../spark/sql/gluten/GlutenFallbackSuite.scala |   6 +-
 .../spark/sql/gluten/GlutenFallbackSuite.scala |   6 +-
 .../spark/sql/gluten/GlutenFallbackSuite.scala |   6 +-
 9 files changed, 556 insertions(+), 371 deletions(-)

diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
index 8f2607a97..1bd0d3b93 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
@@ -20,9 +20,9 @@ import org.apache.gluten.GlutenConfig
 import org.apache.gluten.backendsapi.BackendsApiManager
 import org.apache.gluten.exception.GlutenNotSupportException
 import org.apache.gluten.execution._
-import org.apache.gluten.expression.ExpressionUtils.getExpressionTreeDepth
 import org.apache.gluten.extension.{GlutenPlan, ValidationResult}
 import 
org.apache.gluten.extension.columnar.TransformHints.EncodeTransformableTagImplicits
+import org.apache.gluten.extension.columnar.validator.{Validator, Validators}
 import org.apache.gluten.sql.shims.SparkShimLoader
 
 import org.apache.spark.api.python.EvalPythonExecTransformer
@@ -295,45 +295,16 @@ case class FallbackEmptySchemaRelation() extends 
Rule[SparkPlan] {
 // If false is returned or any unsupported exception is thrown, a row guard 
will
 // be added on the top of that plan to prevent actual conversion.
 case class AddTransformHintRule() extends Rule[SparkPlan] {
-  val columnarConf: GlutenConfig = GlutenConfig.getConf
-  val scanOnly: Boolean = columnarConf.enableScanOnly
-  val enableColumnarShuffle: Boolean =
-!scanOnly && BackendsApiManager.getSettings.supportColumnarShuffleExec()
-  val enableColumnarSort: Boolean = !scanOnly && 
columnarConf.enableColumnarSort
-  val enableColumnarWindow: Boolean = !scanOnly && 
columnarConf.enableColumnarWindow
-  val enableColumnarWindowGroupLimit: Boolean = !scanOnly &&
-columnarConf.enableColumnarWindowGroupLimit
-  val enableColumnarSortMergeJoin: Boolean = !scanOnly &&
-BackendsApiManager.getSettings.supportSortMergeJoinExec()
-  val enableColumnarBatchScan: Boolean = columnarConf.enableColumnarBatchScan
-  val enableColumnarFileScan: Boolean = columnarConf.enableColumnarFileScan
-  val enableColumnarHiveTableScan: Boolean = 
columnarConf.enableColumnarHiveTableScan
-  val enableColumnarProject: Boolean = !scanOnly && 
columnarConf.enableColumnarProject
-  val enableColumnarFilter: Boolean = columnarConf.enableColumnarFilter
-  val fallbackExpressionsThreshold: Int = 
columnarConf.fallbackExpressionsThreshold
-  val enableColumnarHashAgg: Boolean = !scanOnly && 
columnarConf.enableColumnarHashAgg
-  val enableColumnarUnion: Boolean = !scanOnly && 
columnarConf.enableColumnarUnion
-  val enableColumnarExpand: Boolean = !scanOnly && 
columnarConf.enableColumnarExpand
-  val enableColumnarShuffledHashJoin: Boolean =
-!scanOnly && columnarConf.enableColumnarShuffledHashJoin
-  val enableColumnarBroadcastExchange: Boolean = !scanOnly &&
-columnarConf.enableColumnarBroadcastExchange
-  val enableColumnarBroadcastJoin: Boolean = !scanOnly &&
-columnarConf.enableColumnarBroadcastJoin
-  val enableColumnarLimit: Boolean = !scanOnly && 
columnarConf.enableColumnarLimit
-  val enableColumnarGenerate: Boolean = !scanOnly && 
columnarConf.enableColumnarGenerate
-  val enableColumnarCoalesce: Boolean = !scanOnly && 
columnarConf.enableColumnarCoalesce
-  val enableTakeOrderedAndProject: Boolean =
-!scanOnly && columnarConf.enableTakeOrderedAndProject &&
-  enableColumnarSort && enableColumnarLimit && enableColumnarShuffle && 
enableColumnarProject
-  val enableColumnarWrite: Boolean = 

(incubator-gluten) branch main updated (48f481fcb -> dc281ce93)

2024-04-24 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from 48f481fcb [VL] CI: Add TPC-H / TPC-DS job at SF30 with Spark 3.4 
(#5490)
 add dc281ce93 [VL] CI: Minor optimizations for cache build settings

No new revisions were added by this update.

Summary of changes:
 .github/workflows/velox_docker_cache.yml | 11 ++-
 1 file changed, 10 insertions(+), 1 deletion(-)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated (43d72626f -> 7b7cff3f4)

2024-04-23 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from 43d72626f [GLUTEN-5484] add missing tests for clickhouse (#5485)
 add 7b7cff3f4 [VL] Rename Velox backend APIs to make consistent with CH 
(#5464)

No new revisions were added by this update.

Summary of changes:
 .../apache/gluten/backendsapi/velox/VeloxBackend.scala | 18 +-
 ...{BroadcastApiImpl.scala => VeloxBroadcastApi.scala} |  2 +-
 .../{IteratorApiImpl.scala => VeloxIteratorApi.scala}  |  2 +-
 .../{ListenerApiImpl.scala => VeloxListenerApi.scala}  |  6 +++---
 .../{MetricsApiImpl.scala => VeloxMetricsApi.scala}|  2 +-
 ...anExecApiImpl.scala => VeloxSparkPlanExecApi.scala} |  2 +-
 ...nsformerApiImpl.scala => VeloxTransformerApi.scala} |  2 +-
 ...{ValidatorApiImpl.scala => VeloxValidatorApi.scala} |  2 +-
 .../org/apache/spark/sql/expression/UDFResolver.scala  | 12 ++--
 .../apache/spark/util/sketch/VeloxBloomFilterTest.java |  4 ++--
 10 files changed, 26 insertions(+), 26 deletions(-)
 rename 
backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/{BroadcastApiImpl.scala
 => VeloxBroadcastApi.scala} (96%)
 rename 
backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/{IteratorApiImpl.scala
 => VeloxIteratorApi.scala} (99%)
 rename 
backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/{ListenerApiImpl.scala
 => VeloxListenerApi.scala} (98%)
 rename 
backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/{MetricsApiImpl.scala
 => VeloxMetricsApi.scala} (99%)
 rename 
backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/{SparkPlanExecApiImpl.scala
 => VeloxSparkPlanExecApi.scala} (99%)
 rename 
backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/{TransformerApiImpl.scala
 => VeloxTransformerApi.scala} (98%)
 rename 
backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/{ValidatorApiImpl.scala
 => VeloxValidatorApi.scala} (98%)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Fix case-class inheritance for VeloxColumnarWriteFilesExec (#5480)

2024-04-23 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new acbdce981 [VL] Fix case-class inheritance for 
VeloxColumnarWriteFilesExec  (#5480)
acbdce981 is described below

commit acbdce9810186d0aa26ac72f3974a1310e67e6cd
Author: Hongze Zhang 
AuthorDate: Tue Apr 23 18:44:36 2024 +0800

[VL] Fix case-class inheritance for VeloxColumnarWriteFilesExec  (#5480)
---
 .../clickhouse/CHSparkPlanExecApi.scala|  4 +-
 .../backendsapi/velox/SparkPlanExecApiImpl.scala   |  6 +-
 .../execution/VeloxColumnarWriteFilesExec.scala| 64 ++
 .../gluten/backendsapi/SparkPlanExecApi.scala  |  4 +-
 .../org/apache/spark/util/SparkDirectoryUtil.scala |  6 +-
 .../datasources/GlutenV1WriteCommandSuite.scala| 10 ++--
 .../spark/sql/sources/GlutenInsertSuite.scala  |  2 +-
 .../datasources/GlutenV1WriteCommandSuite.scala| 10 ++--
 .../spark/sql/sources/GlutenInsertSuite.scala  |  2 +-
 9 files changed, 74 insertions(+), 34 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 55983171b..7ea12ffe7 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -49,7 +49,7 @@ import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.delta.files.TahoeFileIndex
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec
-import org.apache.spark.sql.execution.datasources.{FileFormat, 
HadoopFsRelation, WriteFilesExec}
+import org.apache.spark.sql.execution.datasources.{FileFormat, 
HadoopFsRelation}
 import 
org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules.NativeWritePostRule
 import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
 import 
org.apache.spark.sql.execution.datasources.v2.clickhouse.source.DeltaMergeTreeFileFormat
@@ -677,7 +677,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
   partitionColumns: Seq[Attribute],
   bucketSpec: Option[BucketSpec],
   options: Map[String, String],
-  staticPartitions: TablePartitionSpec): WriteFilesExec = {
+  staticPartitions: TablePartitionSpec): SparkPlan = {
 throw new GlutenNotSupportException("ColumnarWriteFilesExec is not support 
in ch backend.")
   }
 
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/SparkPlanExecApiImpl.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/SparkPlanExecApiImpl.scala
index 9952147b9..64a2dc2ce 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/SparkPlanExecApiImpl.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/SparkPlanExecApiImpl.scala
@@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.datasources.{FileFormat, WriteFilesExec}
+import org.apache.spark.sql.execution.datasources.FileFormat
 import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, 
ShuffleExchangeExec}
 import org.apache.spark.sql.execution.joins.BuildSideRelation
 import org.apache.spark.sql.execution.metric.SQLMetric
@@ -506,8 +506,8 @@ class SparkPlanExecApiImpl extends SparkPlanExecApi {
   partitionColumns: Seq[Attribute],
   bucketSpec: Option[BucketSpec],
   options: Map[String, String],
-  staticPartitions: TablePartitionSpec): WriteFilesExec = {
-new VeloxColumnarWriteFilesExec(
+  staticPartitions: TablePartitionSpec): SparkPlan = {
+VeloxColumnarWriteFilesExec(
   child,
   fileFormat,
   partitionColumns,
diff --git 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
index 57fbc8104..26d249f90 100644
--- 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
+++ 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/VeloxColumnarWriteFilesExec.scala
@@ -18,6 +18,7 @@ package org.apache.spark.sql.execution
 
 import org.apache.gluten.backendsapi.BackendsApiManager
 import org.apache.gluten.columnarbatch.ColumnarBatches
+import org.apache.gluten.exception.GlutenException
 import org.apache.gluten.extension.GlutenPl

(incubator-gluten) branch main updated: [VL] Avoid using debug instance of JniWorkspace in VeloxBloomFilterTest

2024-04-22 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new c2e77860f [VL] Avoid using debug instance of JniWorkspace in 
VeloxBloomFilterTest
c2e77860f is described below

commit c2e77860ff56dbde5a44159ebef2a96855bf
Author: Hongze Zhang 
AuthorDate: Tue Apr 23 10:04:01 2024 +0800

[VL] Avoid using debug instance of JniWorkspace in VeloxBloomFilterTest
---
 .../gluten/backendsapi/velox/ListenerApiImpl.scala |  2 ++
 .../spark/util/sketch/VeloxBloomFilterTest.java|  2 --
 .../org/apache/gluten/vectorized/JniWorkspace.java |  3 +-
 .../org/apache/spark/util/SparkDirectoryUtil.scala | 33 --
 .../gluten/vectorized/NativePlanEvaluator.java |  3 +-
 .../spark/shuffle/ColumnarShuffleWriter.scala  |  1 +
 6 files changed, 37 insertions(+), 7 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/ListenerApiImpl.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/ListenerApiImpl.scala
index bb3b6ecf0..0ea95ad8e 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/ListenerApiImpl.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/ListenerApiImpl.scala
@@ -31,6 +31,7 @@ import 
org.apache.spark.sql.execution.datasources.velox.{VeloxOrcWriterInjects,
 import org.apache.spark.sql.expression.UDFResolver
 import org.apache.spark.sql.internal.GlutenConfigUtil
 import org.apache.spark.sql.internal.StaticSQLConf
+import org.apache.spark.util.SparkDirectoryUtil
 
 import org.apache.commons.lang3.StringUtils
 
@@ -140,6 +141,7 @@ class ListenerApiImpl extends ListenerApi {
   }
 
   private def initialize(conf: SparkConf): Unit = {
+SparkDirectoryUtil.init(conf)
 val debugJni = conf.getBoolean(GlutenConfig.GLUTEN_DEBUG_MODE, 
defaultValue = false) &&
   conf.getBoolean(GlutenConfig.GLUTEN_DEBUG_KEEP_JNI_WORKSPACE, 
defaultValue = false)
 if (debugJni) {
diff --git 
a/backends-velox/src/test/java/org/apache/spark/util/sketch/VeloxBloomFilterTest.java
 
b/backends-velox/src/test/java/org/apache/spark/util/sketch/VeloxBloomFilterTest.java
index ce439b864..fc1cc03f8 100644
--- 
a/backends-velox/src/test/java/org/apache/spark/util/sketch/VeloxBloomFilterTest.java
+++ 
b/backends-velox/src/test/java/org/apache/spark/util/sketch/VeloxBloomFilterTest.java
@@ -18,7 +18,6 @@ package org.apache.spark.util.sketch;
 
 import org.apache.gluten.backendsapi.ListenerApi;
 import org.apache.gluten.backendsapi.velox.ListenerApiImpl;
-import org.apache.gluten.vectorized.JniWorkspace;
 
 import org.apache.spark.SparkConf;
 import org.apache.spark.util.TaskResources$;
@@ -33,7 +32,6 @@ public class VeloxBloomFilterTest {
 
   @BeforeClass
   public static void setup() {
-JniWorkspace.enableDebug();
 final ListenerApi api = new ListenerApiImpl();
 api.onDriverStart(new SparkConf());
   }
diff --git 
a/gluten-core/src/main/java/org/apache/gluten/vectorized/JniWorkspace.java 
b/gluten-core/src/main/java/org/apache/gluten/vectorized/JniWorkspace.java
index 84edfba18..a7c12387a 100644
--- a/gluten-core/src/main/java/org/apache/gluten/vectorized/JniWorkspace.java
+++ b/gluten-core/src/main/java/org/apache/gluten/vectorized/JniWorkspace.java
@@ -65,7 +65,8 @@ public class JniWorkspace {
   private static JniWorkspace createDefault() {
 try {
   final String tempRoot =
-  SparkDirectoryUtil.namespace("jni")
+  SparkDirectoryUtil.get()
+  .namespace("jni")
   .mkChildDirRandomly(UUID.randomUUID().toString())
   .getAbsolutePath();
   return createOrGet(tempRoot);
diff --git 
a/gluten-core/src/main/scala/org/apache/spark/util/SparkDirectoryUtil.scala 
b/gluten-core/src/main/scala/org/apache/spark/util/SparkDirectoryUtil.scala
index 1d5c21736..435ee8df7 100644
--- a/gluten-core/src/main/scala/org/apache/spark/util/SparkDirectoryUtil.scala
+++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkDirectoryUtil.scala
@@ -16,7 +16,7 @@
  */
 package org.apache.spark.util
 
-import org.apache.spark.SparkEnv
+import org.apache.spark.SparkConf
 import org.apache.spark.internal.Logging
 
 import _root_.org.apache.gluten.exception.GlutenException
@@ -30,8 +30,8 @@ import java.nio.file.Paths
  * Manages Gluten's local directories, for storing jars, libs, spill files, or 
other temporary
  * stuffs.
  */
-object SparkDirectoryUtil extends Logging {
-  private val ROOTS = Utils.getConfiguredLocalDirs(SparkEnv.get.conf).flatMap {
+class SparkDirectoryUtil private (roots: Array[String]) extends Logging {
+  private val ROOTS = roots.flatMap {
 rootDir =>
   try {
 val localDir = Utils.createDirectory(rootDir, "gluten")
@@ -65,6 +65,33 @@ object SparkDirector

(incubator-gluten) branch main updated: [VL] Support fallback processing of velox_bloom_filter_agg (#5477)

2024-04-22 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new a7806e159 [VL] Support fallback processing of velox_bloom_filter_agg 
(#5477)
a7806e159 is described below

commit a7806e159f75315e6b8127fce10caaac6a21d25e
Author: Hongze Zhang 
AuthorDate: Tue Apr 23 10:01:46 2024 +0800

[VL] Support fallback processing of velox_bloom_filter_agg (#5477)
---
 .../apache/spark/util/sketch/VeloxBloomFilter.java |  38 +-
 .../util/sketch/VeloxBloomFilterJniWrapper.java|   8 ++
 .../BloomFilterMightContainJointRewriteRule.scala  |   5 +-
 .../aggregate/VeloxBloomFilterAggregate.scala  |  63 +++---
 .../spark/util/sketch/VeloxBloomFilterTest.java| 136 -
 cpp/velox/jni/VeloxJniWrapper.cc   |  73 ++-
 .../java/org/apache/gluten/exec/RuntimeAware.java  |   4 +
 .../sql/GlutenBloomFilterAggregateQuerySuite.scala |   2 +-
 .../sql/GlutenBloomFilterAggregateQuerySuite.scala |  35 +-
 .../sql/GlutenBloomFilterAggregateQuerySuite.scala |   4 +-
 10 files changed, 339 insertions(+), 29 deletions(-)

diff --git 
a/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilter.java
 
b/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilter.java
index 71051..59716ed79 100644
--- 
a/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilter.java
+++ 
b/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilter.java
@@ -19,6 +19,7 @@ package org.apache.spark.util.sketch;
 import org.apache.commons.io.IOUtils;
 
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -33,6 +34,15 @@ public class VeloxBloomFilter extends BloomFilter {
 handle = jni.init(data);
   }
 
+  private VeloxBloomFilter(int capacity) {
+jni = VeloxBloomFilterJniWrapper.create();
+handle = jni.empty(capacity);
+  }
+
+  public static VeloxBloomFilter empty(int capacity) {
+return new VeloxBloomFilter(capacity);
+  }
+
   public static VeloxBloomFilter readFrom(InputStream in) {
 try {
   byte[] all = IOUtils.toByteArray(in);
@@ -50,6 +60,15 @@ public class VeloxBloomFilter extends BloomFilter {
 }
   }
 
+  public byte[] serialize() {
+try (ByteArrayOutputStream o = new ByteArrayOutputStream()) {
+  writeTo(o);
+  return o.toByteArray();
+} catch (IOException e) {
+  throw new RuntimeException(e);
+}
+  }
+
   @Override
   public double expectedFpp() {
 throw new UnsupportedOperationException("Not yet implemented");
@@ -72,7 +91,8 @@ public class VeloxBloomFilter extends BloomFilter {
 
   @Override
   public boolean putLong(long item) {
-throw new UnsupportedOperationException("Not yet implemented");
+jni.insertLong(handle, item);
+return true;
   }
 
   @Override
@@ -87,7 +107,18 @@ public class VeloxBloomFilter extends BloomFilter {
 
   @Override
   public BloomFilter mergeInPlace(BloomFilter other) throws 
IncompatibleMergeException {
-throw new UnsupportedOperationException("Not yet implemented");
+if (!(other instanceof VeloxBloomFilter)) {
+  throw new IncompatibleMergeException(
+  "Cannot merge Velox bloom-filter with non-Velox bloom-filter");
+}
+final VeloxBloomFilter from = (VeloxBloomFilter) other;
+
+if (!jni.isCompatibleWith(from.jni)) {
+  throw new IncompatibleMergeException(
+  "Cannot merge Velox bloom-filters with different Velox runtimes");
+}
+jni.mergeFrom(handle, from.handle);
+return this;
   }
 
   @Override
@@ -117,6 +148,7 @@ public class VeloxBloomFilter extends BloomFilter {
 
   @Override
   public void writeTo(OutputStream out) throws IOException {
-throw new UnsupportedOperationException("Not yet implemented");
+byte[] data = jni.serialize(handle);
+out.write(data);
   }
 }
diff --git 
a/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilterJniWrapper.java
 
b/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilterJniWrapper.java
index a369c8a30..572e2c7ac 100644
--- 
a/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilterJniWrapper.java
+++ 
b/backends-velox/src/main/java/org/apache/spark/util/sketch/VeloxBloomFilterJniWrapper.java
@@ -36,7 +36,15 @@ public class VeloxBloomFilterJniWrapper implements 
RuntimeAware {
 return runtime.getHandle();
   }
 
+  public native long empty(int capacity);
+
   public native long init(byte[] data);
 
+  public native void insertLong(long handle, long item);
+
   public native boolean mightContainLong(long handle, long item);
+
+  public native void mergeFrom(long handle, long other);
+
+

(incubator-gluten) branch main updated (4af5ea7b0 -> 75623f17d)

2024-04-21 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from 4af5ea7b0 [VL] Refine install libhdfs3 script (#5465)
 add 75623f17d [VL] Daily Update Velox Version (2024_04_21) (#5474)

No new revisions were added by this update.

Summary of changes:
 cpp/velox/compute/WholeStageResultIterator.cc | 3 ++-
 ep/build-velox/src/get_velox.sh   | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Rework co-fallback mechanism of bloom-filter might_contain/agg (#5435)

2024-04-19 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 884ef6470 [VL] Rework co-fallback mechanism of bloom-filter 
might_contain/agg (#5435)
884ef6470 is described below

commit 884ef647034a65e70d6f41e884a9e0d99a10be9b
Author: Hongze Zhang 
AuthorDate: Fri Apr 19 15:39:36 2024 +0800

[VL] Rework co-fallback mechanism of bloom-filter might_contain/agg (#5435)
---
 .../gluten/backendsapi/clickhouse/CHBackend.scala  |   2 +-
 .../clickhouse/CHSparkPlanExecApi.scala|   6 +
 .../apache/spark/util/sketch/VeloxBloomFilter.java | 122 +
 .../util/sketch/VeloxBloomFilterJniWrapper.java|  42 +++
 .../gluten/backendsapi/velox/ListenerApiImpl.scala |  10 +-
 .../backendsapi/velox/SparkPlanExecApiImpl.scala   |  25 +++--
 .../gluten/backendsapi/velox/VeloxBackend.scala|   5 +-
 .../BloomFilterMightContainJointRewriteRule.scala  |  48 
 .../expressions/VeloxBloomFilterMightContain.scala |  98 +
 .../aggregate/VeloxBloomFilterAggregate.scala  |  95 
 .../apache/spark/sql/expression/UDFResolver.scala  |  14 +--
 .../spark/util/sketch/VeloxBloomFilterTest.java|  54 +
 cpp/velox/jni/VeloxJniWrapper.cc   |  32 +-
 .../gluten/backendsapi/BackendSettingsApi.scala|   3 +-
 .../gluten/extension/ColumnarOverrides.scala   |   3 +-
 .../extension/columnar/TransformHintRule.scala |  48 +---
 .../columnar/enumerated/EnumeratedApplier.scala|   3 +-
 .../columnar/heuristic/HeuristicApplier.scala  |   3 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |  14 +--
 .../sql/GlutenBloomFilterAggregateQuerySuite.scala |  15 ++-
 .../sql/GlutenBloomFilterAggregateQuerySuite.scala |   4 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |  13 ++-
 .../sql/GlutenBloomFilterAggregateQuerySuite.scala |  46 +++-
 .../scala/org/apache/gluten/GlutenConfig.scala |   2 +-
 .../org/apache/gluten/sql/shims/SparkShims.scala   |  36 +-
 .../gluten/sql/shims/spark32/Spark32Shims.scala|  34 +-
 .../gluten/sql/shims/spark33/Spark33Shims.scala|  80 ++
 .../gluten/sql/shims/spark34/Spark34Shims.scala|  83 +-
 .../gluten/sql/shims/spark35/Spark35Shims.scala|  82 +-
 29 files changed, 838 insertions(+), 184 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
index 7217a979b..276ce11fb 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
@@ -260,7 +260,7 @@ object CHBackendSettings extends BackendSettingsApi with 
Logging {
 
   override def requiredInputFilePaths(): Boolean = true
 
-  override def enableBloomFilterAggFallbackRule(): Boolean = false
+  override def requireBloomFilterAggMightContainJointFallback(): Boolean = 
false
 
   def maxShuffleReadRows(): Long = {
 SparkEnv.get.conf
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index fb564c9e2..55983171b 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -25,6 +25,7 @@ import 
org.apache.gluten.expression.ConverterUtils.FunctionConfig
 import org.apache.gluten.extension.{CountDistinctWithoutExpand, 
FallbackBroadcastHashJoin, FallbackBroadcastHashJoinPrepQueryStage, 
RewriteToDateExpresstionRule}
 import org.apache.gluten.extension.columnar.AddTransformHintRule
 import 
org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides
+import org.apache.gluten.sql.shims.SparkShimLoader
 import org.apache.gluten.substrait.expression.{ExpressionBuilder, 
ExpressionNode, WindowFunctionNode}
 import org.apache.gluten.utils.CHJoinValidateUtil
 import org.apache.gluten.vectorized.CHColumnarBatchSerializer
@@ -624,6 +625,11 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
   override def genExtendedStrategies(): List[SparkSession => Strategy] =
 List()
 
+  /** Define backend specfic expression mappings. */
+  override def extraExpressionMappings: Seq[Sig] = {
+SparkShimLoader.getSparkShims.bloomFilterExpressionMappings()
+  }
+
   override def genStringTranslateTransformer(
   substraitExprName: String,
   srcExpr: ExpressionTransformer,
diff --git 
a/backends-velox/src/main/java/org/apache/spark/u

(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_19) (#5452)

2024-04-18 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new c4693444e [VL] Daily Update Velox Version (2024_04_19) (#5452)
c4693444e is described below

commit c4693444e71f2ddf0e29bdffccd12c39b0b1b931
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Fri Apr 19 13:08:01 2024 +0800

[VL] Daily Update Velox Version (2024_04_19) (#5452)

Signed-off-by: glutenperfbot 
Co-authored-by: glutenperfbot 
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index 10f385a9b..ba4ef44fe 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_18
+VELOX_BRANCH=2024_04_19
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated (fb987c911 -> 3e5742a6b)

2024-04-18 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from fb987c911 [VL] Daily Update Velox Version (2024_04_18) (#5443)
 add 3e5742a6b [GLUTEN-5251][VL] Fix inconsistency of the default value for 
spark.gluten.sql.columnar.backend.velox.maxSpillFileSize (#5450)

No new revisions were added by this update.

Summary of changes:
 cpp/velox/compute/VeloxBackend.cc | 2 +-
 cpp/velox/compute/WholeStageResultIterator.cc | 2 +-
 docs/get-started/Velox.md | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_18) (#5443)

2024-04-18 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new fb987c911 [VL] Daily Update Velox Version (2024_04_18) (#5443)
fb987c911 is described below

commit fb987c911bd3e6569272c5745b90e0d7d57d6912
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Thu Apr 18 15:29:06 2024 +0800

[VL] Daily Update Velox Version (2024_04_18) (#5443)

Signed-off-by: glutenperfbot 
Co-authored-by: glutenperfbot 
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index b6637924c..10f385a9b 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_17
+VELOX_BRANCH=2024_04_18
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Add a bad test case when bloom_filter_agg is fallen back while might_contain is not

2024-04-17 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new a1db28ddf [VL] Add a bad test case when bloom_filter_agg is fallen 
back while might_contain is not
a1db28ddf is described below

commit a1db28ddfa8f54ab7bff4f60d4dbfb734cc94d26
Author: Hongze Zhang 
AuthorDate: Wed Apr 17 15:46:06 2024 +0800

[VL] Add a bad test case when bloom_filter_agg is fallen back while 
might_contain is not
---
 .../extension/columnar/TransformHintRule.scala |  2 +-
 .../gluten/utils/velox/VeloxTestSettings.scala |  1 +
 .../sql/GlutenBloomFilterAggregateQuerySuite.scala | 33 ++
 3 files changed, 35 insertions(+), 1 deletion(-)

diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
index d19a82050..1559ba8b3 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/TransformHintRule.scala
@@ -324,7 +324,7 @@ case class FallbackBloomFilterAggIfNeeded() extends 
Rule[SparkPlan] {
 case a: org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec 
=>
   tagNotTransformableRecursive(a.executedPlan)
 case _ =>
-  p.children.map(tagNotTransformableRecursive)
+  p.children.foreach(tagNotTransformableRecursive)
   }
 }
 
diff --git 
a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
 
b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
index 79f182e24..518908c9c 100644
--- 
a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
+++ 
b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
@@ -46,6 +46,7 @@ import 
org.apache.spark.sql.sources.{GlutenBucketedReadWithoutHiveSupportSuite,
 class VeloxTestSettings extends BackendTestSettings {
   enableSuite[GlutenStringFunctionsSuite]
   enableSuite[GlutenBloomFilterAggregateQuerySuite]
+.excludeGlutenTest("Test bloom_filter_agg fallback with might_contain 
offloaded")
   enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite]
   enableSuite[GlutenDataSourceV2DataFrameSuite]
   enableSuite[GlutenDataSourceV2FunctionSuite]
diff --git 
a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala
 
b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala
index c12cf8217..7a4a4b427 100644
--- 
a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala
+++ 
b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala
@@ -113,4 +113,37 @@ class GlutenBloomFilterAggregateQuerySuite
   }
 }
   }
+
+  testGluten("Test bloom_filter_agg fallback with might_contain offloaded") {
+val table = "bloom_filter_test"
+val numEstimatedItems = 500L
+val numBits = GlutenConfig.getConf.veloxBloomFilterMaxNumBits
+val sqlString = s"""
+   |SELECT col positive_membership_test
+   |FROM $table
+   |WHERE might_contain(
+   |(SELECT bloom_filter_agg(col,
+   |  cast($numEstimatedItems as long),
+   |  cast($numBits as long))
+   | FROM $table), col)
+  """.stripMargin
+
+withTempView(table) {
+  (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 20L))
+.toDF("col")
+.createOrReplaceTempView(table)
+  withSQLConf(
+GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key -> "false"
+  ) {
+val df = spark.sql(sqlString)
+df.collect
+assert(
+  collectWithSubqueries(df.queryExecution.executedPlan) {
+case h if h.isInstanceOf[HashAggregateExecBaseTransformer] => h
+  }.isEmpty,
+  df.queryExecution.executedPlan
+)
+  }
+}
+  }
 }


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated (fe4c70f22 -> cb4ae9abb)

2024-04-16 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from fe4c70f22 [VL] Fallback window operator when the range frame contain 
literal (#5431)
 add cb4ae9abb [VL] Daily Update Velox Version (2024_04_17) (#5429)

No new revisions were added by this update.

Summary of changes:
 cpp/velox/CMakeLists.txt| 3 +++
 ep/build-velox/src/get_velox.sh | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [CORE] Move memory off-heap conf checks to driver plugin (#5128)

2024-04-16 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new b8e4c0935 [CORE] Move memory off-heap conf checks to driver plugin 
(#5128)
b8e4c0935 is described below

commit b8e4c093587b79715bf228fd8a2098fd023bea74
Author: Zhen Wang <643348...@qq.com>
AuthorDate: Wed Apr 17 10:07:11 2024 +0800

[CORE] Move memory off-heap conf checks to driver plugin (#5128)
---
 .../scala/org/apache/gluten/GlutenPlugin.scala | 24 ++
 1 file changed, 11 insertions(+), 13 deletions(-)

diff --git a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala 
b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
index 5c9c56b39..972b6ddba 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
@@ -143,10 +143,18 @@ private[gluten] class GlutenDriverPlugin extends 
DriverPlugin with Logging {
 }
 conf.set(SPARK_SESSION_EXTS_KEY, extensions)
 
-// off-heap bytes
-if (!conf.contains(GlutenConfig.GLUTEN_OFFHEAP_SIZE_KEY)) {
-  throw new GlutenException(s"${GlutenConfig.GLUTEN_OFFHEAP_SIZE_KEY} is 
not set")
+// check memory off-heap enabled and size
+val minOffHeapSize = "1MB"
+if (
+  !conf.getBoolean(GlutenConfig.GLUTEN_OFFHEAP_ENABLED, false) ||
+  conf.getSizeAsBytes(GlutenConfig.GLUTEN_OFFHEAP_SIZE_KEY, 0) < 
JavaUtils.byteStringAsBytes(
+minOffHeapSize)
+) {
+  throw new GlutenException(
+s"Must set '${GlutenConfig.GLUTEN_OFFHEAP_ENABLED}' to true " +
+  s"and set '${GlutenConfig.GLUTEN_OFFHEAP_SIZE_KEY}' to be greater 
than $minOffHeapSize")
 }
+
 // Session's local time zone must be set. If not explicitly set by user, 
its default
 // value (detected for the platform) is used, consistent with spark.
 conf.set(GLUTEN_DEFAULT_SESSION_TIMEZONE_KEY, 
SQLConf.SESSION_LOCAL_TIMEZONE.defaultValueString)
@@ -202,16 +210,6 @@ private[gluten] class GlutenExecutorPlugin extends 
ExecutorPlugin {
   override def init(ctx: PluginContext, extraConf: util.Map[String, String]): 
Unit = {
 val conf = ctx.conf()
 
-// Must set the 'spark.memory.offHeap.size' value to native memory malloc
-if (
-  !conf.getBoolean("spark.memory.offHeap.enabled", false) ||
-  (JavaUtils.byteStringAsBytes(
-conf.get("spark.memory.offHeap.size").toString) / 1024 / 1024).toInt 
<= 0
-) {
-  throw new IllegalArgumentException(
-s"Must set 'spark.memory.offHeap.enabled' to true" +
-  s" and set off heap memory size by option 
'spark.memory.offHeap.size'")
-}
 // Initialize Backends API
 // TODO categorize the APIs by driver's or executor's
 BackendsApiManager.initialize()


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_16) (#5413)

2024-04-16 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 9e39cb90b [VL] Daily Update Velox Version (2024_04_16) (#5413)
9e39cb90b is described below

commit 9e39cb90b2dbcb0d518c79386fde8db5b9857275
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Tue Apr 16 16:45:38 2024 +0800

[VL] Daily Update Velox Version (2024_04_16) (#5413)

Signed-off-by: glutenperfbot 
Co-authored-by: glutenperfbot 
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index bf4272fef..18b02dbcf 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_15
+VELOX_BRANCH=2024_04_16
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated (3e80977bf -> 607068bd7)

2024-04-16 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from 3e80977bf [GLUTEN-5249] [CH] fix throw Unexpected empty column when 
reading csv file (#5254)
 add 607068bd7 [VL] Add independent operator for top-n processing in 
TakeOrderedAndProjectExecTransformer (#5409)

No new revisions were added by this update.

Summary of changes:
 .../backendsapi/velox/SparkPlanExecApiImpl.scala   |  17 ++-
 .../apache/gluten/execution/TopNTransformer.scala  | 125 +
 .../apache/gluten/execution/VeloxTPCHSuite.scala   |   1 +
 cpp/velox/substrait/SubstraitToVeloxPlan.cc|  39 ++-
 cpp/velox/substrait/SubstraitToVeloxPlan.h |   6 +-
 .../substrait/SubstraitToVeloxPlanValidator.cc |  55 ++---
 .../substrait/SubstraitToVeloxPlanValidator.h  |   3 +
 docs/developers/SubstraitModifications.md  |   1 +
 .../apache/gluten/substrait/rel/RelBuilder.java|  17 +++
 .../rel/{SortRelNode.java => TopNNode.java}|  33 +++---
 .../substrait/proto/substrait/algebra.proto|  10 ++
 .../gluten/backendsapi/SparkPlanExecApi.scala  |   2 +
 .../apache/gluten/execution/LimitTransformer.scala |  22 +---
 .../TakeOrderedAndProjectExecTransformer.scala |  14 ++-
 14 files changed, 190 insertions(+), 155 deletions(-)
 copy 
gluten-core/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala
 => 
backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala 
(52%)
 copy 
gluten-core/src/main/java/org/apache/gluten/substrait/rel/{SortRelNode.java => 
TopNNode.java} (70%)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Move out a non-common test case from VeloxTPCHSuite (#5402)

2024-04-14 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 1facff721 [VL] Move out a non-common test case from VeloxTPCHSuite 
(#5402)
1facff721 is described below

commit 1facff7211087c3ee7d5f65615c933cb2520437b
Author: Hongze Zhang 
AuthorDate: Mon Apr 15 12:58:32 2024 +0800

[VL] Move out a non-common test case from VeloxTPCHSuite (#5402)
---
 .../apache/gluten/execution/VeloxTPCHSuite.scala   | 34 +-
 1 file changed, 20 insertions(+), 14 deletions(-)

diff --git 
a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala
 
b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala
index ca7aced51..49ec776e4 100644
--- 
a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala
+++ 
b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala
@@ -244,8 +244,27 @@ abstract class VeloxTPCHSuite extends 
VeloxTPCHTableSupport {
   checkGoldenFile(_, 22)
 }
   }
+}
+
+class VeloxTPCHDistinctSpillSuite extends VeloxTPCHTableSupport {
+  override protected def sparkConf: SparkConf = {
+super.sparkConf
+  .set("spark.memory.offHeap.size", "50m")
+  .set("spark.gluten.memory.overAcquiredMemoryRatio", "0.9") // to trigger 
distinct spill early
+  }
 
-  test("test 'order by limit'") {
+  test("distinct spill") {
+val df = spark.sql("select count(distinct *) from lineitem limit 1")
+TestUtils.compareAnswers(df.collect(), Seq(Row(60175)))
+  }
+}
+
+class VeloxTPCHMiscSuite extends VeloxTPCHTableSupport {
+  override protected def sparkConf: SparkConf = {
+super.sparkConf
+  }
+
+  test("order by limit") {
 val df = spark.sql(
   """
 |select n_nationkey from nation order by n_nationkey limit 5
@@ -261,19 +280,6 @@ abstract class VeloxTPCHSuite extends 
VeloxTPCHTableSupport {
   }
 }
 
-class VeloxTPCHDistinctSpill extends VeloxTPCHTableSupport {
-  override protected def sparkConf: SparkConf = {
-super.sparkConf
-  .set("spark.memory.offHeap.size", "50m")
-  .set("spark.gluten.memory.overAcquiredMemoryRatio", "0.9") // to trigger 
distinct spill early
-  }
-
-  test("distinct spill") {
-val df = spark.sql("select count(distinct *) from lineitem limit 1")
-TestUtils.compareAnswers(df.collect(), Seq(Row(60175)))
-  }
-}
-
 class VeloxTPCHV1Suite extends VeloxTPCHSuite {
   override def subType(): String = "v1"
 


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_15) (#5400)

2024-04-14 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 962610254 [VL] Daily Update Velox Version (2024_04_15) (#5400)
962610254 is described below

commit 96261025468d27be86563b6c52be2dda5f989642
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Mon Apr 15 11:12:21 2024 +0800

[VL] Daily Update Velox Version (2024_04_15) (#5400)

Signed-off-by: glutenperfbot 
Co-authored-by: glutenperfbot 
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index 63594dae0..bf4272fef 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_14
+VELOX_BRANCH=2024_04_15
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [GLUTEN-5381] Refine testWithSpecifiedSparkVersion to compare major version (#5382)

2024-04-14 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 0d78ac2fc [GLUTEN-5381] Refine testWithSpecifiedSparkVersion to 
compare major version (#5382)
0d78ac2fc is described below

commit 0d78ac2fcc8e81fdd2bf069bd973c65a3382e4c6
Author: Yan Ma 
AuthorDate: Mon Apr 15 09:55:10 2024 +0800

[GLUTEN-5381] Refine testWithSpecifiedSparkVersion to compare major version 
(#5382)
---
 .../execution/ScalarFunctionsValidateSuite.scala   |  4 +--
 .../org/apache/spark/sql/GlutenQueryTest.scala | 34 +-
 .../apache/gluten/execution/VeloxDeltaSuite.scala  |  8 ++---
 3 files changed, 32 insertions(+), 14 deletions(-)

diff --git 
a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
 
b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
index 6fceb7364..daf8fa269 100644
--- 
a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
+++ 
b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
@@ -550,7 +550,7 @@ class ScalarFunctionsValidateSuite extends 
FunctionsValidateTest {
 }
   }
 
-  testWithSpecifiedSparkVersion("Test url_decode function", Some("3.4.2")) {
+  testWithSpecifiedSparkVersion("Test url_decode function", Some("3.4")) {
 withTempPath {
   path =>
 Seq("https%3A%2F%2Fspark.apache.org")
@@ -565,7 +565,7 @@ class ScalarFunctionsValidateSuite extends 
FunctionsValidateTest {
 }
   }
 
-  testWithSpecifiedSparkVersion("Test url_encode function", Some("3.4.2")) {
+  testWithSpecifiedSparkVersion("Test url_encode function", Some("3.4")) {
 withTempPath {
   path =>
 Seq("https://spark.apache.org;)
diff --git 
a/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala 
b/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala
index d6a2a0c63..93ae5cd36 100644
--- a/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala
+++ b/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala
@@ -44,14 +44,35 @@ abstract class GlutenQueryTest extends PlanTest {
 
   protected def spark: SparkSession
 
+  def shouldRun(
+  minSparkVersion: Option[String] = None,
+  maxSparkVersion: Option[String] = None): Boolean = {
+val version = SPARK_VERSION_SHORT.split("\\.")
+var shouldRun = true
+if (!minSparkVersion.isEmpty) {
+  val minVersion = minSparkVersion.get.split("\\.");
+  shouldRun =
+minVersion(0) < version(0) || (minVersion(0) == version(0) && 
minVersion(1) <= version(1))
+  if (!maxSparkVersion.isEmpty) {
+val maxVersion = maxSparkVersion.get.split("\\.")
+shouldRun = shouldRun && (maxVersion(0) > version(0) || maxVersion(0) 
== version(
+  0) && maxVersion(1) >= version(1))
+  }
+} else {
+  if (!maxSparkVersion.isEmpty) {
+val maxVersion = maxSparkVersion.get.split("\\.")
+shouldRun =
+  maxVersion(0) > version(0) || maxVersion(0) == version(0) && 
maxVersion(1) >= version(1)
+  }
+}
+shouldRun
+  }
+
   def ignore(
   testName: String,
   minSparkVersion: Option[String] = None,
   maxSparkVersion: Option[String] = None)(testFun: => Any): Unit = {
-if (
-  minSparkVersion.forall(_ <= SPARK_VERSION_SHORT)
-  && maxSparkVersion.forall(_ >= SPARK_VERSION_SHORT)
-) {
+if (shouldRun(minSparkVersion, maxSparkVersion)) {
   ignore(testName) {
 testFun
   }
@@ -62,10 +83,7 @@ abstract class GlutenQueryTest extends PlanTest {
   testName: String,
   minSparkVersion: Option[String] = None,
   maxSparkVersion: Option[String] = None)(testFun: => Any): Unit = {
-if (
-  minSparkVersion.forall(_ <= SPARK_VERSION_SHORT)
-  && maxSparkVersion.forall(_ >= SPARK_VERSION_SHORT)
-) {
+if (shouldRun(minSparkVersion, maxSparkVersion)) {
   test(testName) {
 testFun
   }
diff --git 
a/gluten-delta/src/test/scala/org/apache/gluten/execution/VeloxDeltaSuite.scala 
b/gluten-delta/src/test/scala/org/apache/gluten/execution/VeloxDeltaSuite.scala
index 9347f7fa5..8cf4baae8 100644
--- 
a/gluten-delta/src/test/scala/org/apache/gluten/execution/VeloxDeltaSuite.scala
+++ 
b/gluten-delta/src/test/scala/org/apache/gluten/execution/VeloxDeltaSuite.scala
@@ -43,7 +43,7 @@ class VeloxDeltaSuite extends WholeStageTransformerSuite {
 
   // IdMapping is supported in Delta 2.2 (related to Spark3.3.1)
   // Disable for Spark3.5.
-  te

(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_14) (#5397)

2024-04-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 0e3387255 [VL] Daily Update Velox Version (2024_04_14) (#5397)
0e3387255 is described below

commit 0e33872558a401edc202a768189c87e9034fac94
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Sun Apr 14 11:04:01 2024 +0800

[VL] Daily Update Velox Version (2024_04_14) (#5397)

Signed-off-by: glutenperfbot 
Co-authored-by: glutenperfbot 
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index ed40ccb8a..63594dae0 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_13
+VELOX_BRANCH=2024_04_14
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_12) (#5375)

2024-04-12 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new fd06368f3 [VL] Daily Update Velox Version (2024_04_12) (#5375)
fd06368f3 is described below

commit fd06368f3f1b3ac0495b22129e2562ada842da6b
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Fri Apr 12 19:23:03 2024 +0800

[VL] Daily Update Velox Version (2024_04_12) (#5375)

Signed-off-by: glutenperfbot 
Co-authored-by: glutenperfbot 
Co-authored-by: Hongze Zhang 
---
 .github/workflows/velox_docker.yml |  3 ++-
 ep/build-velox/src/get_velox.sh|  2 +-
 .../scala/org/apache/gluten/GlutenConfig.scala |  9 +++
 .../integration/tpc/command/Parameterized.java | 28 --
 .../integration/tpc/action/Parameterized.scala | 17 +
 5 files changed, 50 insertions(+), 9 deletions(-)

diff --git a/.github/workflows/velox_docker.yml 
b/.github/workflows/velox_docker.yml
index 07960813e..8515d79d2 100644
--- a/.github/workflows/velox_docker.yml
+++ b/.github/workflows/velox_docker.yml
@@ -245,7 +245,8 @@ jobs:
 -d=OFFHEAP_SIZE:6g,spark.memory.offHeap.size=6g \
 -d=OFFHEAP_SIZE:4g,spark.memory.offHeap.size=4g \
 
-d=OVER_ACQUIRE:0.3,spark.gluten.memory.overAcquiredMemoryRatio=0.3 \
--d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5
+
-d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5 \
+--excluded-dims=OFFHEAP_SIZE:4g
   - name: TPC-DS SF30.0 Parquet local spark3.2 Q67 low memory, memory 
isolation on
 run: |
   cd tools/gluten-it \
diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index fa6deb374..32e33ed48 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_11
+VELOX_BRANCH=2024_04_12
 VELOX_HOME=""
 
 #Set on run gluten on HDFS
diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala 
b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala
index bc98ea25e..cfaf54cdd 100644
--- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala
+++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala
@@ -256,6 +256,8 @@ class GlutenConfig(conf: SQLConf) extends Logging {
 
   def veloxSpillStrategy: String = conf.getConf(COLUMNAR_VELOX_SPILL_STRATEGY)
 
+  def veloxMaxSpillLevel: Int = conf.getConf(COLUMNAR_VELOX_MAX_SPILL_LEVEL)
+
   def veloxMaxSpillFileSize: Long = 
conf.getConf(COLUMNAR_VELOX_MAX_SPILL_FILE_SIZE)
 
   def veloxSpillFileSystem: String = 
conf.getConf(COLUMNAR_VELOX_SPILL_FILE_SYSTEM)
@@ -1249,6 +1251,13 @@ object GlutenConfig {
   .checkValues(Set("none", "auto"))
   .createWithDefault("auto")
 
+  val COLUMNAR_VELOX_MAX_SPILL_LEVEL =
+buildConf("spark.gluten.sql.columnar.backend.velox.maxSpillLevel")
+  .internal()
+  .doc("The max allowed spilling level with zero being the initial 
spilling level")
+  .intConf
+  .createWithDefault(4)
+
   val COLUMNAR_VELOX_MAX_SPILL_FILE_SIZE =
 buildConf("spark.gluten.sql.columnar.backend.velox.maxSpillFileSize")
   .internal()
diff --git 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
index 7c9c64399..1f94cb256 100644
--- 
a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
+++ 
b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/tpc/command/Parameterized.java
@@ -16,8 +16,10 @@
  */
 package org.apache.gluten.integration.tpc.command;
 
+import com.google.common.base.Preconditions;
 import org.apache.gluten.integration.tpc.TpcMixin;
 import org.apache.gluten.integration.tpc.action.Dim;
+import org.apache.gluten.integration.tpc.action.DimKv;
 import org.apache.gluten.integration.tpc.action.DimValue;
 import org.apache.commons.lang3.ArrayUtils;
 import picocli.CommandLine;
@@ -25,12 +27,7 @@ import scala.Tuple2;
 import scala.collection.JavaConverters;
 import scala.collection.Seq;
 
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -64,13 +61,30 @@ public class Parameterized implements Callable {
   @CommandLine.Option(names = {"-d", "--dim"}, descri

(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_11) (#5360)

2024-04-11 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 154f8750e [VL] Daily Update Velox Version (2024_04_11) (#5360)
154f8750e is described below

commit 154f8750e1f9cd93c4b5362bbca2db219193e01f
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Thu Apr 11 17:38:52 2024 +0800

[VL] Daily Update Velox Version (2024_04_11) (#5360)
---
 ep/build-velox/src/build_velox.sh|  3 ++-
 ep/build-velox/src/get_velox.sh  |  2 +-
 .../scala/org/apache/spark/sql/GlutenQueryTest.scala | 16 +++-
 .../org/apache/gluten/execution/VeloxIcebergSuite.scala  |  9 ++---
 .../apache/gluten/execution/VeloxTPCHIcebergSuite.scala  |  5 +
 5 files changed, 29 insertions(+), 6 deletions(-)

diff --git a/ep/build-velox/src/build_velox.sh 
b/ep/build-velox/src/build_velox.sh
index 47423f335..a38107c8f 100755
--- a/ep/build-velox/src/build_velox.sh
+++ b/ep/build-velox/src/build_velox.sh
@@ -114,7 +114,8 @@ function compile {
 fi
   fi
 
-  COMPILE_OPTION="-DVELOX_ENABLE_PARQUET=ON -DVELOX_BUILD_TESTING=OFF"
+  CXX_FLAGS='-Wno-missing-field-initializers'
+  COMPILE_OPTION="-DCMAKE_CXX_FLAGS=\"$CXX_FLAGS\" -DVELOX_ENABLE_PARQUET=ON 
-DVELOX_BUILD_TESTING=OFF"
   if [ $BUILD_TEST_UTILS == "ON" ]; then
   COMPILE_OPTION="$COMPILE_OPTION -DVELOX_BUILD_TEST_UTILS=ON"
   fi
diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index f4c2e4313..fa6deb374 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_10
+VELOX_BRANCH=2024_04_11
 VELOX_HOME=""
 
 #Set on run gluten on HDFS
diff --git 
a/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala 
b/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala
index ae83a56d1..d6a2a0c63 100644
--- a/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala
+++ b/gluten-core/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala
@@ -26,7 +26,7 @@ import org.apache.spark.SPARK_VERSION_SHORT
 import org.apache.spark.rpc.GlutenDriverEndpoint
 import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort, 
Subquery, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.execution.SQLExecution
 import org.apache.spark.sql.execution.columnar.InMemoryRelation
@@ -44,6 +44,20 @@ abstract class GlutenQueryTest extends PlanTest {
 
   protected def spark: SparkSession
 
+  def ignore(
+  testName: String,
+  minSparkVersion: Option[String] = None,
+  maxSparkVersion: Option[String] = None)(testFun: => Any): Unit = {
+if (
+  minSparkVersion.forall(_ <= SPARK_VERSION_SHORT)
+  && maxSparkVersion.forall(_ >= SPARK_VERSION_SHORT)
+) {
+  ignore(testName) {
+testFun
+  }
+}
+  }
+
   def testWithSpecifiedSparkVersion(
   testName: String,
   minSparkVersion: Option[String] = None,
diff --git 
a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala
 
b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala
index d9abd6472..9ffcc80cf 100644
--- 
a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala
+++ 
b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala
@@ -58,7 +58,8 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite {
 }
   }
 
-  testWithSpecifiedSparkVersion("iceberg bucketed join", Some("3.4")) {
+  // Ignored due to failures, see 
https://github.com/apache/incubator-gluten/issues/5362
+  ignore("iceberg bucketed join", Some("3.4")) {
 val leftTable = "p_str_tb"
 val rightTable = "p_int_tb"
 withTable(leftTable, rightTable) {
@@ -138,7 +139,8 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite {
 }
   }
 
-  testWithSpecifiedSparkVersion("iceberg bucketed join with partition", 
Some("3.4")) {
+  // Ignored due to failures, see 
https://github.com/apache/incubator-gluten/issues/5362
+  ignore("iceberg bucketed join with partition", Some("3.4")) {
 val leftTable = "p_str_tb"
 val rightTable = "p_int_tb"
 withTable(leftTable, rightTable) {
@@ -218,7 +220,8 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite {
 }
   }
 
-  testWithSpecifiedSparkVersio

(incubator-gluten) branch main updated: [CORE][VL] Avoid re-exploring explored nodes in DpPlanner (#5363)

2024-04-11 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new bc99c4910 [CORE][VL] Avoid re-exploring explored nodes in DpPlanner 
(#5363)
bc99c4910 is described below

commit bc99c4910e26648195fec789c090b06c2b4379e2
Author: Hongze Zhang 
AuthorDate: Thu Apr 11 15:53:26 2024 +0800

[CORE][VL] Avoid re-exploring explored nodes in DpPlanner (#5363)
---
 .../src/main/scala/org/apache/gluten/ras/Ras.scala | 14 ++---
 .../scala/org/apache/gluten/ras/RasCluster.scala   |  8 +--
 .../main/scala/org/apache/gluten/ras/RasNode.scala | 21 
 .../scala/org/apache/gluten/ras/RasPlanner.scala   | 10 ++--
 .../org/apache/gluten/ras/best/BestFinder.scala|  6 +--
 .../org/apache/gluten/ras/dp/DpClusterAlgo.scala   |  2 +-
 .../org/apache/gluten/ras/dp/DpGroupAlgo.scala |  2 +-
 .../scala/org/apache/gluten/ras/dp/DpPlanner.scala | 62 +-
 .../scala/org/apache/gluten/ras/memo/Memo.scala|  6 +--
 .../org/apache/gluten/ras/path/OutputFilter.scala  | 15 ++
 .../org/apache/gluten/ras/rule/EnforcerRule.scala  | 39 +-
 .../org/apache/gluten/ras/rule/RuleApplier.scala   | 10 ++--
 .../apache/gluten/ras/vis/GraphvizVisualizer.scala |  4 +-
 .../scala/org/apache/gluten/ras/RasSuite.scala | 53 ++
 14 files changed, 184 insertions(+), 68 deletions(-)

diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
index f3d46847e..804d04d81 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
@@ -171,7 +171,7 @@ class Ras[T <: AnyRef] private (
 
   private[ras] def isInfCost(cost: Cost) = 
costModel.costComparator().equiv(cost, infCost)
 
-  private[ras] def toUnsafeKey(node: T): UnsafeKey[T] = UnsafeKey(this, node)
+  private[ras] def toHashKey(node: T): UnsafeHashKey[T] = UnsafeHashKey(this, 
node)
 }
 
 object Ras {
@@ -251,15 +251,17 @@ object Ras {
 }
   }
 
-  trait UnsafeKey[T]
+  trait UnsafeHashKey[T]
 
-  private object UnsafeKey {
-def apply[T <: AnyRef](ras: Ras[T], self: T): UnsafeKey[T] = new 
UnsafeKeyImpl(ras, self)
-private class UnsafeKeyImpl[T <: AnyRef](ras: Ras[T], val self: T) extends 
UnsafeKey[T] {
+  private object UnsafeHashKey {
+def apply[T <: AnyRef](ras: Ras[T], self: T): UnsafeHashKey[T] =
+  new UnsafeHashKeyImpl(ras, self)
+private class UnsafeHashKeyImpl[T <: AnyRef](ras: Ras[T], val self: T)
+  extends UnsafeHashKey[T] {
   override def hashCode(): Int = ras.planModel.hashCode(self)
   override def equals(other: Any): Boolean = {
 other match {
-  case that: UnsafeKeyImpl[T] => ras.planModel.equals(self, that.self)
+  case that: UnsafeHashKeyImpl[T] => ras.planModel.equals(self, 
that.self)
   case _ => false
 }
   }
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasCluster.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasCluster.scala
index 1b30e1242..eb2b41a91 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasCluster.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasCluster.scala
@@ -16,7 +16,7 @@
  */
 package org.apache.gluten.ras
 
-import org.apache.gluten.ras.Ras.UnsafeKey
+import org.apache.gluten.ras.Ras.UnsafeHashKey
 import org.apache.gluten.ras.memo.MemoTable
 import org.apache.gluten.ras.property.PropertySet
 
@@ -55,16 +55,16 @@ object RasCluster {
 override val ras: Ras[T],
 metadata: Metadata)
   extends MutableRasCluster[T] {
-  private val deDup: mutable.Set[UnsafeKey[T]] = mutable.Set()
+  private val deDup: mutable.Set[UnsafeHashKey[T]] = mutable.Set()
   private val buffer: mutable.ListBuffer[CanonicalNode[T]] =
 mutable.ListBuffer()
 
   override def contains(t: CanonicalNode[T]): Boolean = {
-deDup.contains(t.toUnsafeKey())
+deDup.contains(t.toHashKey())
   }
 
   override def add(t: CanonicalNode[T]): Unit = {
-val key = t.toUnsafeKey()
+val key = t.toHashKey()
 assert(!deDup.contains(key))
 ras.metadataModel.verify(metadata, 
ras.metadataModel.metadataOf(t.self()))
 deDup += key
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala
index 65ff8b735..710a4e682 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala
@@ -16,7 +16,7 @@
  */
 package org.apache.gluten.ras
 
-import org.apache.gluten.ras.Ras.UnsafeKey
+import org.apache.gluten.ras.R

(incubator-gluten) branch main updated: [CORE][VL] RAS: Pattern matching by node classes

2024-04-10 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 449ac3259 [CORE][VL] RAS: Pattern matching by node classes
449ac3259 is described below

commit 449ac32590ece0b1b1d451467906a4a44c911f44
Author: Hongze Zhang 
AuthorDate: Thu Apr 11 12:02:19 2024 +0800

[CORE][VL] RAS: Pattern matching by node classes
---
 .../org/apache/gluten/ras/dp/DpClusterAlgo.scala   |  2 +-
 .../scala/org/apache/gluten/ras/dp/DpPlanner.scala |  2 +-
 .../apache/gluten/ras/memo/ForwardMemoTable.scala  | 10 ++--
 .../scala/org/apache/gluten/ras/memo/Memo.scala| 43 ++---
 .../org/apache/gluten/ras/memo/MemoTable.scala | 56 +++---
 .../scala/org/apache/gluten/ras/path/Pattern.scala | 15 ++
 .../scala/org/apache/gluten/ras/rule/Shape.scala   | 16 +--
 .../apache/gluten/ras/vis/GraphvizVisualizer.scala |  8 +++-
 .../org/apache/gluten/ras/rule/PatternSuite.scala  | 48 +++
 9 files changed, 133 insertions(+), 67 deletions(-)

diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpClusterAlgo.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpClusterAlgo.scala
index e90ba448b..6fd95772b 100644
--- 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpClusterAlgo.scala
+++ 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpClusterAlgo.scala
@@ -81,7 +81,7 @@ object DpClusterAlgo {
 }
 
 override def browseX(x: InClusterNode[T]): Iterable[RasClusterKey] = {
-  val allGroups = memoTable.allGroups()
+  val allGroups = memoTable.asGroupSupplier()
   x.can
 .getChildrenGroups(allGroups)
 .map(gn => allGroups(gn.groupId()).clusterKey())
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala
index 8acf66c59..4a9e3f0f0 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala
@@ -60,7 +60,7 @@ private class DpPlanner[T <: AnyRef] private (
   }
 
   private def findBest(memoTable: MemoTable[T], groupId: Int): Best[T] = {
-val cKey = memoTable.allGroups()(groupId).clusterKey()
+val cKey = memoTable.asGroupSupplier()(groupId).clusterKey()
 val algoDef = new DpExploreAlgoDef[T]
 val adjustment = new ExploreAdjustment(ras, memoTable, rules, 
enforcerRuleSet)
 DpClusterAlgo.resolve(memoTable, algoDef, adjustment, cKey)
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/ForwardMemoTable.scala
 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/ForwardMemoTable.scala
index 0895544d5..dd4033866 100644
--- 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/ForwardMemoTable.scala
+++ 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/ForwardMemoTable.scala
@@ -155,11 +155,13 @@ class ForwardMemoTable[T <: AnyRef] private (override val 
ras: Ras[T])
 groupBuffer(id)
   }
 
-  override def allClusters(): Seq[RasClusterKey] = clusterKeyBuffer
+  override def allClusterKeys(): Seq[RasClusterKey] = clusterKeyBuffer
 
-  override def allGroups(): Seq[RasGroup[T]] = groupBuffer
-
-  override def allDummyGroups(): Seq[RasGroup[T]] = dummyGroupBuffer
+  override def allGroupIds(): Seq[Int] = {
+val from = -dummyGroupBuffer.size
+val to = groupBuffer.size
+(from until to).toVector
+  }
 
   private def ancestorClusterIdOf(key: RasClusterKey): Int = {
 clusterDisjointSet.find(key.id())
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala
index c1bb0a6bf..6406b8fb1 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala
@@ -18,7 +18,6 @@ package org.apache.gluten.ras.memo
 
 import org.apache.gluten.ras._
 import org.apache.gluten.ras.Ras.UnsafeKey
-import org.apache.gluten.ras.RasCluster.ImmutableRasCluster
 import org.apache.gluten.ras.property.PropertySet
 import org.apache.gluten.ras.vis.GraphvizVisualizer
 
@@ -78,7 +77,7 @@ object Memo {
   memoTable.getDummyGroup(clusterKey)
 }
 
-private def toCacheKeyUnsafe(n: T): MemoCacheKey[T] = {
+private def toCacheKey(n: T): MemoCacheKey[T] = {
   MemoCacheKey(ras, n)
 }
 
@@ -90,11 +89,11 @@ object Memo {
 
   val childrenPrepares = ras.planModel.childrenOf(n).map(child => 
prepareInsert(child))
 
-  val canUnsafe = ras.withNewChildren(
+  val keyUnsafe = ras.withNewChildren(
 n,
 childrenPrepares.map(childPrepare => 
dummyGroupOf(childPrepare.clusterKey()).self()))
 
-   

(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_10) (#5347)

2024-04-09 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new aa5c45c21 [VL] Daily Update Velox Version (2024_04_10) (#5347)
aa5c45c21 is described below

commit aa5c45c211ed67ea68a9e76c8e56f02d61e26457
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Wed Apr 10 13:05:52 2024 +0800

[VL] Daily Update Velox Version (2024_04_10) (#5347)
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index 9b9442882..f4c2e4313 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_09
+VELOX_BRANCH=2024_04_10
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_09) (#5328)

2024-04-09 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new c3613d24f [VL] Daily Update Velox Version (2024_04_09) (#5328)
c3613d24f is described below

commit c3613d24fe36bedc0d20a4d6abd557f5d0494258
Author: Hongze Zhang 
AuthorDate: Tue Apr 9 14:14:33 2024 +0800

[VL] Daily Update Velox Version (2024_04_09) (#5328)
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index 7c72b21ca..9b9442882 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_08
+VELOX_BRANCH=2024_04_09
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [CORE][VL] RAS: Group expansion support (#5323)

2024-04-08 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 19da3bb1c [CORE][VL] RAS: Group expansion support (#5323)
19da3bb1c is described below

commit 19da3bb1cc8677913c097d7c4ccd1fc17189c438
Author: Hongze Zhang 
AuthorDate: Mon Apr 8 18:16:45 2024 +0800

[CORE][VL] RAS: Group expansion support (#5323)
---
 .../src/main/scala/org/apache/gluten/ras/Ras.scala |   6 +-
 .../main/scala/org/apache/gluten/ras/RasNode.scala |   2 +-
 .../org/apache/gluten/ras/best/BestFinder.scala|   2 -
 .../gluten/ras/best/GroupBasedBestFinder.scala |   2 +-
 .../scala/org/apache/gluten/ras/dp/DpPlanner.scala |  23 ++--
 .../gluten/ras/exaustive/ExhaustivePlanner.scala   |  21 ++--
 .../apache/gluten/ras/memo/ForwardMemoTable.scala  |  25 +++--
 .../scala/org/apache/gluten/ras/memo/Memo.scala|  28 +++--
 .../org/apache/gluten/ras/memo/MemoTable.scala |  17 ++-
 .../org/apache/gluten/ras/path/OutputFilter.scala  |  33 --
 .../org/apache/gluten/ras/path/OutputWizard.scala  | 100 +++---
 .../org/apache/gluten/ras/path/PathFinder.scala|  35 +--
 .../scala/org/apache/gluten/ras/path/Pattern.scala |   9 +-
 .../org/apache/gluten/ras/rule/RuleApplier.scala   |  12 +--
 .../scala/org/apache/gluten/ras/rule/Shape.scala   |  10 ++
 .../org/apache/gluten/ras/PropertySuite.scala  | 116 -
 .../scala/org/apache/gluten/ras/RasSuite.scala |  67 
 .../scala/org/apache/gluten/ras/RasSuiteBase.scala |   2 +-
 .../org/apache/gluten/ras/mock/MockMemoState.scala |   5 +
 .../org/apache/gluten/ras/mock/MockRasPath.scala   |   2 +-
 .../apache/gluten/ras/path/PathFinderSuite.scala   |  60 ++-
 .../org/apache/gluten/ras/path/WizardSuite.scala   |  14 +++
 .../gluten/ras/specific/DistributedSuite.scala |   2 +
 23 files changed, 449 insertions(+), 144 deletions(-)

diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
index 9910fab6f..f3d46847e 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
@@ -31,7 +31,7 @@ trait Optimization[T <: AnyRef] {
   constraintSet: PropertySet[T],
   altConstraintSets: Seq[PropertySet[T]]): RasPlanner[T]
 
-  def propSetsOf(plan: T): PropertySet[T]
+  def propSetOf(plan: T): PropertySet[T]
 
   def withNewConfig(confFunc: RasConfig => RasConfig): Optimization[T]
 }
@@ -49,7 +49,7 @@ object Optimization {
 
   implicit class OptimizationImplicits[T <: AnyRef](opt: Optimization[T]) {
 def newPlanner(plan: T): RasPlanner[T] = {
-  opt.newPlanner(plan, opt.propSetsOf(plan), List.empty)
+  opt.newPlanner(plan, opt.propSetOf(plan), List.empty)
 }
 def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] = {
   opt.newPlanner(plan, constraintSet, List.empty)
@@ -131,7 +131,7 @@ class Ras[T <: AnyRef] private (
 RasPlanner(this, altConstraintSets, constraintSet, plan)
   }
 
-  override def propSetsOf(plan: T): PropertySet[T] = 
propertySetFactory().get(plan)
+  override def propSetOf(plan: T): PropertySet[T] = 
propertySetFactory().get(plan)
 
   private[ras] def withNewChildren(node: T, newChildren: Seq[T]): T = {
 val oldChildren = planModel.childrenOf(node)
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala
index 878020391..65ff8b735 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasNode.scala
@@ -54,7 +54,7 @@ trait CanonicalNode[T <: AnyRef] extends RasNode[T] {
 object CanonicalNode {
   def apply[T <: AnyRef](ras: Ras[T], canonical: T): CanonicalNode[T] = {
 assert(ras.isCanonical(canonical))
-val propSet = ras.propSetsOf(canonical)
+val propSet = ras.propSetOf(canonical)
 val children = ras.planModel.childrenOf(canonical)
 new CanonicalNodeImpl[T](ras, canonical, propSet, children.size)
   }
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/best/BestFinder.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/best/BestFinder.scala
index 0912ab536..90a0adfb2 100644
--- 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/best/BestFinder.scala
+++ 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/best/BestFinder.scala
@@ -52,12 +52,10 @@ object BestFinder {
 
   private[best] def newBest[T <: AnyRef](
   ras: Ras[T],
-  allGroups: Seq[RasGroup[T]],
   group: RasGroup[T],
   groupToCosts: Map[Int, KnownCostGroup[T]]): Best[T] = {
 
 val bestPath = groupToCosts(group.id()).bes

(incubator-gluten) branch main updated: [VL] RAS: Group reduction support

2024-04-07 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 446749bd3 [VL] RAS: Group reduction support
446749bd3 is described below

commit 446749bd3f62503d1974b77bf7ca52870919f280
Author: Hongze Zhang 
AuthorDate: Mon Apr 8 13:52:52 2024 +0800

[VL] RAS: Group reduction support
---
 .github/workflows/velox_docker.yml |  1 +
 .../scala/org/apache/gluten/ras/memo/Memo.scala| 16 ++---
 .../scala/org/apache/gluten/ras/RasSuite.scala | 26 ++
 3 files changed, 40 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/velox_docker.yml 
b/.github/workflows/velox_docker.yml
index 32c4ccc95..afadfa5d2 100644
--- a/.github/workflows/velox_docker.yml
+++ b/.github/workflows/velox_docker.yml
@@ -24,6 +24,7 @@ on:
   - 'gluten-celeborn/common'
   - 'gluten-celeborn/package'
   - 'gluten-celeborn/velox'
+  - 'gluten-ras/**'
   - 'gluten-core/**'
   - 'gluten-data/**'
   - 'gluten-delta/**'
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala
index 66626b756..49281a82d 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/memo/Memo.scala
@@ -126,8 +126,20 @@ object Memo {
   extends MemoLike[T] {
   private val ras = parent.ras
 
+  // TODO: Traverse up the tree to do more merges.
   private def prepareInsert(node: T): Prepare[T] = {
-assert(!ras.isGroupLeaf(node))
+if (ras.isGroupLeaf(node)) {
+  val group = 
parent.memoTable.allGroups()(ras.planModel.getGroupId(node))
+  val residentCluster = group.clusterKey()
+
+  if (residentCluster == targetCluster) {
+return Prepare.cluster(parent, targetCluster)
+  }
+  // The resident cluster of group leaf is not the same with target 
cluster.
+  // Merge.
+  parent.memoTable.mergeClusters(residentCluster, targetCluster)
+  return Prepare.cluster(parent, targetCluster)
+}
 
 val childrenPrepares =
   ras.planModel.childrenOf(node).map(child => 
parent.prepareInsert(child))
@@ -155,8 +167,6 @@ object Memo {
 }
 // The new node already memorized to memo, but in the different 
cluster.
 // Merge the two clusters.
-//
-// TODO: Traverse up the tree to do more merges.
 parent.memoTable.mergeClusters(cachedCluster, targetCluster)
 Prepare.tree(parent, targetCluster, childrenPrepares)
   }
diff --git 
a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/RasSuite.scala 
b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/RasSuite.scala
index 0ad825181..f8a3d0799 100644
--- a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/RasSuite.scala
+++ b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/RasSuite.scala
@@ -172,6 +172,32 @@ abstract class RasSuite extends AnyFunSuite {
 assert(optimized == Unary(23, Unary(23, Leaf(70
   }
 
+  test(s"Group reduction") {
+object RemoveUnary extends RasRule[TestNode] {
+  override def shift(node: TestNode): Iterable[TestNode] = node match {
+case Unary(cost, child) => List(child)
+case other => List.empty
+  }
+
+  override def shape(): Shape[TestNode] = Shapes.fixedHeight(1)
+}
+
+val ras =
+  Ras[TestNode](
+PlanModelImpl,
+CostModelImpl,
+MetadataModelImpl,
+PropertyModelImpl,
+ExplainImpl,
+RasRule.Factory.reuse(List(RemoveUnary)))
+.withNewConfig(_ => conf)
+val plan = Unary(60, Unary(90, Leaf(70)))
+val planner = ras.newPlanner(plan)
+val optimized = planner.plan()
+
+assert(optimized == Leaf(70))
+  }
+
   test(s"Unary node insertion") {
 object InsertUnary2 extends RasRule[TestNode] {
   override def shift(node: TestNode): Iterable[TestNode] = node match {


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [CORE][VL] RAS: Refactor memo cache to look up on cluster-canonical node rather than on group-canonical node (#5305)

2024-04-07 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 993e96afe [CORE][VL] RAS: Refactor memo cache to look up on 
cluster-canonical node rather than on group-canonical node (#5305)
993e96afe is described below

commit 993e96afe81ff85e2928151b4a9b9d45baf4b79f
Author: Hongze Zhang 
AuthorDate: Mon Apr 8 11:42:51 2024 +0800

[CORE][VL] RAS: Refactor memo cache to look up on cluster-canonical node 
rather than on group-canonical node (#5305)
---
 .../planner/property/GlutenPropertyModel.scala |   2 +
 .../org/apache/gluten/ras/MetadataModel.scala  |   3 +-
 .../org/apache/gluten/ras/PropertyModel.scala  |   1 +
 .../src/main/scala/org/apache/gluten/ras/Ras.scala |  48 -
 .../scala/org/apache/gluten/ras/RasCluster.scala   |  16 +-
 .../main/scala/org/apache/gluten/ras/RasNode.scala |  53 -
 .../scala/org/apache/gluten/ras/RasPlanner.scala   |  23 ++-
 .../org/apache/gluten/ras/best/BestFinder.scala|  21 +-
 .../gluten/ras/best/GroupBasedBestFinder.scala |  23 ++-
 .../org/apache/gluten/ras/dp/DpClusterAlgo.scala   |   2 +-
 .../org/apache/gluten/ras/dp/DpGroupAlgo.scala |   2 +-
 .../scala/org/apache/gluten/ras/dp/DpPlanner.scala |   4 +-
 .../org/apache/gluten/ras/dp/DpZipperAlgo.scala|   2 -
 .../gluten/ras/exaustive/ExhaustivePlanner.scala   |  24 ++-
 .../apache/gluten/ras/memo/ForwardMemoTable.scala  |  19 +-
 .../scala/org/apache/gluten/ras/memo/Memo.scala| 214 +
 .../org/apache/gluten/ras/memo/MemoTable.scala |   1 +
 .../scala/org/apache/gluten/ras/path/RasPath.scala |  43 ++---
 .../org/apache/gluten/ras/rule/RuleApplier.scala   |  51 ++---
 .../scala/org/apache/gluten/ras/util/NodeMap.scala |  60 --
 .../apache/gluten/ras/vis/GraphvizVisualizer.scala |   4 +-
 .../org/apache/gluten/ras/OperationSuite.scala |   8 +-
 .../org/apache/gluten/ras/PropertySuite.scala  |  34 +++-
 .../scala/org/apache/gluten/ras/RasSuite.scala |   9 +-
 .../org/apache/gluten/ras/path/RasPathSuite.scala  |  60 --
 .../gluten/ras/specific/DistributedSuite.scala |   4 +
 26 files changed, 453 insertions(+), 278 deletions(-)

diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala
index 54f4e3b84..07dd3fe02 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala
@@ -53,6 +53,8 @@ object GlutenProperties {
 val conv = getProperty(plan)
 plan.children.map(_ => conv)
 }
+
+override def any(): Convention = Conventions.ANY
   }
 
   case class ConventionEnforcerRule(reqConv: Convention) extends 
RasRule[SparkPlan] {
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/MetadataModel.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/MetadataModel.scala
index d2056746c..a81ac31cb 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/MetadataModel.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/MetadataModel.scala
@@ -21,8 +21,9 @@ package org.apache.gluten.ras
  */
 trait MetadataModel[T <: AnyRef] {
   def metadataOf(node: T): Metadata
-  def dummy(): Metadata
   def verify(one: Metadata, other: Metadata): Unit
+
+  def dummy(): Metadata
 }
 
 trait Metadata {}
diff --git 
a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/PropertyModel.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/PropertyModel.scala
index e2ba99136..e764631e7 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/PropertyModel.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/PropertyModel.scala
@@ -26,6 +26,7 @@ trait Property[T <: AnyRef] {
 }
 
 trait PropertyDef[T <: AnyRef, P <: Property[T]] {
+  def any(): P
   def getProperty(plan: T): P
   def getChildrenConstraints(constraint: Property[T], plan: T): Seq[P]
 }
diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala 
b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
index 6832d07c5..9910fab6f 100644
--- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
+++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala
@@ -79,9 +79,10 @@ class Ras[T <: AnyRef] private (
   ruleFactory)
   }
 
-  // Normal groups start with ID 0, so it's safe to use -1 to do validation.
+  private val propSetFactory: PropertySetFactory[T] = 
PropertySetFactory(propertyModel, planModel)
+  // Normal groups start with ID 0, so it's safe to use Int.MinValue to do 
validation.
   private val dummyGroup: T =
-planModel.newGroupLeaf(-1, m

(incubator-gluten) branch main updated: [VL] Daily Update Velox Version (2024_04_08) (#5313)

2024-04-07 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new e10ee1bca [VL] Daily Update Velox Version (2024_04_08) (#5313)
e10ee1bca is described below

commit e10ee1bca1471a178732ec748d9c2857660a3a86
Author: Gluten Performance Bot 
<137994563+glutenperf...@users.noreply.github.com>
AuthorDate: Mon Apr 8 10:51:10 2024 +0800

[VL] Daily Update Velox Version (2024_04_08) (#5313)
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index 04bcabfc1..73b8be76c 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -17,7 +17,7 @@
 set -exu
 
 VELOX_REPO=https://github.com/oap-project/velox.git
-VELOX_BRANCH=2024_04_07
+VELOX_BRANCH=2024_04_08
 VELOX_HOME=""
 
 #Set on run gluten on HDFS


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Re-enable some failed CI jobs

2024-04-06 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 2378d1fd6 [VL] Re-enable some failed CI jobs
2378d1fd6 is described below

commit 2378d1fd6a8149af688a02f625827c52470fdae9
Author: Hongze Zhang 
AuthorDate: Sun Apr 7 13:33:45 2024 +0800

[VL] Re-enable some failed CI jobs
---
 .github/workflows/velox_docker.yml | 224 +++--
 1 file changed, 137 insertions(+), 87 deletions(-)

diff --git a/.github/workflows/velox_docker.yml 
b/.github/workflows/velox_docker.yml
index 1e74a159c..32c4ccc95 100644
--- a/.github/workflows/velox_docker.yml
+++ b/.github/workflows/velox_docker.yml
@@ -193,87 +193,137 @@ jobs:
 --local --preset=velox --benchmark-type=ds --error-on-memleak 
--off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \
 --extra-conf=spark.gluten.sql.ras.enabled=true 
 
-  # run-tpc-test-centos8-oom-randomkill:
-  #   needs: build-native-lib
-  #   strategy:
-  # fail-fast: false
-  # matrix:
-  #   spark: ["spark-3.2"]
-  #   runs-on: ubuntu-20.04
-  #   container: centos:8
-  #   steps:
-  # - uses: actions/checkout@v2
-  # - name: Download All Artifacts
-  #   uses: actions/download-artifact@v2
-  #   with:
-  # name: velox-native-lib-${{github.sha}}
-  # path: ./cpp/build/releases
-  # - name: Update mirror list
-  #   run: |
-  # sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* 
|| true
-  # sed -i -e 
"s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" 
/etc/yum.repos.d/CentOS-* || true
-  # - name: Setup java and maven
-  #   run: |
-  # yum update -y && yum install -y java-1.8.0-openjdk-devel wget
-  # wget 
https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz
-  # tar -xvf apache-maven-3.8.8-bin.tar.gz
-  # mv apache-maven-3.8.8 /usr/lib/maven
-  # - name: Build for Spark ${{ matrix.spark }}
-  #   run: |
-  # cd $GITHUB_WORKSPACE/ 
-  # export MAVEN_HOME=/usr/lib/maven
-  # export PATH=${PATH}:${MAVEN_HOME}/bin
-  # mvn -ntp clean install -P${{ matrix.spark }} -Pbackends-velox 
-DskipTests
-  # - name: TPC-DS SF30.0 Parquet local spark3.2 Q67/Q95 low memory, 
memory isolation off
-  #   run: |
-  # export MAVEN_HOME=/usr/lib/maven
-  # export PATH=${PATH}:${MAVEN_HOME}/bin
-  # cd tools/gluten-it && \
-  # mvn -ntp clean install -Pspark-3.2 \
-  # && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \
-  #   --local --preset=velox --benchmark-type=ds --error-on-memleak 
--queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \
-  #   --skip-data-gen -m=OffHeapExecutionMemory \
-  #   -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \
-  #   -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \
-  #   -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \
-  #   
-d=OVER_ACQUIRE:0.3,spark.gluten.memory.overAcquiredMemoryRatio=0.3 \
-  #   
-d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5
-  # - name: (To be fixed) TPC-DS SF30.0 Parquet local spark3.2 Q67/Q95 low 
memory, memory isolation on
-  #   run: |
-  # export MAVEN_HOME=/usr/lib/maven
-  # export PATH=${PATH}:${MAVEN_HOME}/bin
-  # cd tools/gluten-it && \
-  # mvn -ntp clean install -Pspark-3.2 \
-  # && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \
-  #   --local --preset=velox --benchmark-type=ds --error-on-memleak 
--queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \
-  #   --skip-data-gen -m=OffHeapExecutionMemory \
-  #   
-d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1
 \
-  #   -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \
-  #   -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \
-  #   
-d=OVER_ACQUIRE:0.3,spark.gluten.memory.overAcquiredMemoryRatio=0.3 \
-  #   
-d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5 || true
-  # - name: TPC-DS SF30.0 Parquet local spark3.2 Q23A/Q23B low memory
-  #   run: |
-  # cd tools/gluten-it && \
-  # GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \
-  #   --local --preset=velox --benchmark-type=ds --error-on-memleak 
--queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 
\
-  #   --skip-data-gen -m=OffHeapExecutionMemory \
-  #   -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \
-  #   
-d=IS

(incubator-gluten) branch main updated: [VL] RAS: Add EnumeratedApplier to manage columnar rule applications when ras is enabled (#5276)

2024-04-06 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new dc4bc629a [VL] RAS: Add EnumeratedApplier to manage columnar rule 
applications when ras is enabled (#5276)
dc4bc629a is described below

commit dc4bc629aaadbdbd285f6d0654643733a8542607
Author: Hongze Zhang 
AuthorDate: Sun Apr 7 12:06:02 2024 +0800

[VL] RAS: Add EnumeratedApplier to manage columnar rule applications when 
ras is enabled (#5276)
---
 .../org/apache/gluten/planner/VeloxRasSuite.scala  |   2 +-
 .../gluten/extension/ColumnarOverrides.scala   | 265 +
 .../columnar/ColumnarRuleApplier.scala}|  10 +-
 .../extension/columnar/ColumnarTransitions.scala   |  38 ++-
 .../{transform => }/ImplementSingleNode.scala  |   3 +-
 .../extension/columnar/MiscColumnarRules.scala |   3 +-
 .../columnar/RewriteSparkPlanRulesManager.scala|  18 +-
 .../columnar/enumerated/EnumeratedApplier.scala| 173 ++
 .../{ => enumerated}/EnumeratedTransform.scala |   4 +-
 .../columnar/heuristic/HeuristicApplier.scala  | 173 ++
 .../extension/columnar/util/AdaptiveContext.scala  |  93 
 .../apache/gluten/planner/GlutenOptimization.scala |  11 -
 .../GlutenFormatWriterInjectsBase.scala|   5 +-
 .../sql/execution/FallbackStrategiesSuite.scala|  13 +-
 .../sql/execution/FallbackStrategiesSuite.scala|  13 +-
 .../sql/execution/FallbackStrategiesSuite.scala|  13 +-
 16 files changed, 534 insertions(+), 303 deletions(-)

diff --git 
a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala 
b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
index 4b1ee935f..cf600ff56 100644
--- 
a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
+++ 
b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
@@ -95,7 +95,7 @@ class VeloxRasSuite extends SharedSparkSession {
 
 object VeloxRasSuite {
   def newRas(): Ras[SparkPlan] = {
-GlutenOptimization().asInstanceOf[Ras[SparkPlan]]
+GlutenOptimization(List()).asInstanceOf[Ras[SparkPlan]]
   }
 
   def newRas(RasRules: Seq[RasRule[SparkPlan]]): Ras[SparkPlan] = {
diff --git 
a/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala
 
b/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala
index b664aa988..8ba51342a 100644
--- 
a/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala
+++ 
b/gluten-core/src/main/scala/org/apache/gluten/extension/ColumnarOverrides.scala
@@ -17,12 +17,10 @@
 package org.apache.gluten.extension
 
 import org.apache.gluten.{GlutenConfig, GlutenSparkExtensionsInjector}
-import org.apache.gluten.backendsapi.BackendsApiManager
 import org.apache.gluten.extension.columnar._
-import 
org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow,
 RemoveTopmostColumnarToRow, TransformPostOverrides, TransformPreOverrides}
-import org.apache.gluten.extension.columnar.transform._
-import org.apache.gluten.metrics.GlutenTimeMetric
-import org.apache.gluten.utils.{LogLevelUtil, PhysicalPlanSelector, PlanUtil}
+import org.apache.gluten.extension.columnar.enumerated.EnumeratedApplier
+import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier
+import org.apache.gluten.utils.LogLevelUtil
 
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.internal.Logging
@@ -30,68 +28,11 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{SparkSession, SparkSessionExtensions}
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule}
+import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.adaptive._
 import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.util.SparkRuleUtil
-
-import scala.collection.mutable.ListBuffer
-
-private[extension] object ColumnarToRowLike {
-  def unapply(plan: SparkPlan): Option[SparkPlan] = {
-plan match {
-  case c2r: ColumnarToRowTransition =>
-Some(c2r.child)
-  case _ => None
-}
-  }
-}
-// This rule will try to add RowToColumnarExecBase and ColumnarToRowExec
-// to support vanilla columnar operators.
-case class InsertColumnarToColumnarTransitions(session: SparkSession) extends 
Rule[SparkPlan] {
-  @transient private val planChangeLogger = new PlanChangeLogger[SparkPlan]()
-
-  private def replaceWithVanillaColumnarToRow(p: SparkPlan): SparkPlan = 
p.transformUp {
-case plan if PlanUtil.isGlutenColumnarOp(plan) =>
-  plan.withNewChildren(plan.children.map {
-   

(incubator-gluten) branch main updated: [VL] CI: Update dependency cache only when main branch is updated (#5234)

2024-04-02 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new cae9bba20 [VL] CI: Update dependency cache only when main branch is 
updated (#5234)
cae9bba20 is described below

commit cae9bba20127852f0fc5fd2d4bf1a17170e7abde
Author: Hongze Zhang 
AuthorDate: Wed Apr 3 10:17:51 2024 +0800

[VL] CI: Update dependency cache only when main branch is updated (#5234)
---
 .github/workflows/velox_docker.yml   | 17 +++-
 .github/workflows/velox_docker_cache.yml | 44 
 dev/ci-velox-buildstatic.sh  |  9 +++
 3 files changed, 57 insertions(+), 13 deletions(-)

diff --git a/.github/workflows/velox_docker.yml 
b/.github/workflows/velox_docker.yml
index 597fca081..793abf78e 100644
--- a/.github/workflows/velox_docker.yml
+++ b/.github/workflows/velox_docker.yml
@@ -52,26 +52,17 @@ jobs:
   - uses: actions/checkout@v2
   - name: Generate cache key
 run: |
-  echo ${{ hashFiles('./ep/build-velox/src/**', './dev/**', './cpp/*') 
}} > cache-key
+  echo ${{ hashFiles('./ep/build-velox/src/**', './dev/**', './cpp/*', 
'./github/workflows/*') }} > cache-key
   - name: Cache
 id: cache
-uses: actions/cache@v3
+uses: actions/cache/restore@v3
 with:
   path: ./cpp/build/releases/
   key: cache-velox-build-${{ hashFiles('./cache-key') }}
-  - name: Build Gluten velox third party
+  - name: Build Gluten Velox third party
 if: ${{ steps.cache.outputs.cache-hit != 'true' }}
 run: |
-  yum install sudo patch java-1.8.0-openjdk-devel -y && \
-  cd $GITHUB_WORKSPACE/ep/build-velox/src && \
-  ./get_velox.sh && \
-  source /opt/rh/devtoolset-9/enable && \
-  source $GITHUB_WORKSPACE//dev/vcpkg/env.sh && \
-  cd $GITHUB_WORKSPACE/ && \
-  sed -i '/^headers/d' ep/build-velox/build/velox_ep/CMakeLists.txt && 
\
-  export NUM_THREADS=4
-  ./dev/builddeps-veloxbe.sh --build_tests=OFF --build_benchmarks=OFF 
--enable_s3=ON \
-  --enable_gcs=ON --enable_hdfs=ON --enable_abfs=ON
+  source dev/ci-velox-buildstatic.sh
   - uses: actions/upload-artifact@v2
 with:
   path: ./cpp/build/releases/
diff --git a/.github/workflows/velox_docker_cache.yml 
b/.github/workflows/velox_docker_cache.yml
new file mode 100644
index 0..25a63781b
--- /dev/null
+++ b/.github/workflows/velox_docker_cache.yml
@@ -0,0 +1,44 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+name: Velox backend Dependency Cache
+
+on:
+  push:
+branches:
+  - 'main'
+
+concurrency:
+  group: ${{ github.repository }}-${{ github.workflow }}
+  cancel-in-progress: true
+
+jobs:
+  cache-native-lib:
+runs-on: ubuntu-20.04
+container: inteldpo/gluten-centos-packaging:latest # centos7 with 
dependencies installed
+steps:
+  - uses: actions/checkout@v2
+  - name: Generate cache key
+run: |
+  echo ${{ hashFiles('./ep/build-velox/src/**', './dev/**', './cpp/*', 
'./github/workflows/*') }} > cache-key
+  - name: Build Gluten Velox third party
+run: |
+  source dev/ci-velox-buildstatic.sh
+  - name: Cache
+id: cache
+uses: actions/cache/save@v3
+with:
+  path: ./cpp/build/releases/
+  key: cache-velox-build-${{ hashFiles('./cache-key') }}
diff --git a/dev/ci-velox-buildstatic.sh b/dev/ci-velox-buildstatic.sh
new file mode 100755
index 0..a9b9d2c3f
--- /dev/null
+++ b/dev/ci-velox-buildstatic.sh
@@ -0,0 +1,9 @@
+yum install sudo patch java-1.8.0-openjdk-devel -y
+cd $GITHUB_WORKSPACE/ep/build-velox/src
+./get_velox.sh
+source /opt/rh/devtoolset-9/enable
+source $GITHUB_WORKSPACE//dev/vcpkg/env.sh
+cd $GITHUB_WORKSPACE/
+sed -i '/^headers/d' ep/build-velox/build/velox_ep/CMakeLists.txt
+export NUM_THREADS=4
+./dev/builddeps-veloxbe.sh --build_tests=OFF 

(incubator-gluten) branch main updated: Revert "[VL] gluten-it: Shorten table creation and query runner logs" (#5237)

2024-04-01 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 7c56d429b Revert "[VL] gluten-it: Shorten table creation and query 
runner logs" (#5237)
7c56d429b is described below

commit 7c56d429b93a0d570afbf8d48e4185756d8c6ade
Author: Hongze Zhang 
AuthorDate: Tue Apr 2 09:14:13 2024 +0800

Revert "[VL] gluten-it: Shorten table creation and query runner logs" 
(#5237)
---
 .../apache/gluten/integration/tpc/TpcRunner.scala  | 15 +---
 .../scala/org/apache/spark/sql/QueryRunner.scala   | 28 ++
 .../apache/spark/sql/SparkSessionSwitcher.scala| 19 +++
 3 files changed, 26 insertions(+), 36 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
index c7ab7febd..ab76dc68c 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
@@ -17,6 +17,7 @@
 package org.apache.gluten.integration.tpc
 
 import org.apache.spark.sql.{AnalysisException, QueryRunner, RunResult, 
SparkSession}
+
 import com.google.common.base.Preconditions
 import org.apache.commons.io.FileUtils
 
@@ -46,14 +47,13 @@ class TpcRunner(val queryResourceFolder: String, val 
dataPath: String) {
 
 object TpcRunner {
   def createTables(spark: SparkSession, dataPath: String): Unit = {
-print("Creating catalog tables: ")
-try {
-  val files = new File(dataPath).listFiles()
-  files.foreach(file => {
+val files = new File(dataPath).listFiles()
+files.foreach(
+  file => {
 if (spark.catalog.tableExists(file.getName)) {
-  print(s"${file.getName}(exists), ")
+  println("Table exists: " + file.getName)
 } else {
-  print(s"${file.getName}, ")
+  println("Creating catalog table: " + file.getName)
   spark.catalog.createTable(file.getName, file.getAbsolutePath, 
"parquet")
   try {
 spark.catalog.recoverPartitions(file.getName)
@@ -62,9 +62,6 @@ object TpcRunner {
   }
 }
   })
-} finally {
-  println("... Done.")
-}
   }
 
   private def delete(path: String): Unit = {
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala 
b/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala
index 20c8a0617..a4044c925 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala
@@ -18,12 +18,7 @@ package org.apache.spark.sql
 
 import org.apache.spark.{SparkContext, Success, TaskKilled}
 import org.apache.spark.executor.ExecutorMetrics
-import org.apache.spark.scheduler.{
-  SparkListener,
-  SparkListenerExecutorMetricsUpdate,
-  SparkListenerTaskEnd,
-  SparkListenerTaskStart
-}
+import org.apache.spark.scheduler.{SparkListener, 
SparkListenerExecutorMetricsUpdate, SparkListenerTaskEnd, 
SparkListenerTaskStart}
 import org.apache.spark.sql.KillTaskListener.INIT_WAIT_TIME_MS
 
 import com.google.common.base.Preconditions
@@ -50,7 +45,8 @@ object QueryRunner {
 "ProcessTreePythonVMemory",
 "ProcessTreePythonRSSMemory",
 "ProcessTreeOtherVMemory",
-"ProcessTreeOtherRSSMemory")
+"ProcessTreeOtherRSSMemory"
+  )
 
   def runTpcQuery(
   spark: SparkSession,
@@ -80,7 +76,7 @@ object QueryRunner {
 }
 killTaskListener.foreach(sc.addSparkListener(_))
 
-print(s"Executing SQL query from resource path $queryPath... ")
+println(s"Executing SQL query from resource path $queryPath...")
 try {
   val sql = resourceToString(queryPath)
   val prev = System.nanoTime()
@@ -94,13 +90,13 @@ object QueryRunner {
   RunResult(rows, millis, collectedMetrics)
 } finally {
   sc.removeSparkListener(metricsListener)
-  killTaskListener.foreach(l => {
-sc.removeSparkListener(l)
-println(s"Successful kill rate ${"%.2f%%"
-  .format(100 * l.successfulKillRate())} during execution of app: 
${sc.applicationId}")
-  })
+  killTaskListener.foreach(
+l => {
+  sc.removeSparkListener(l)
+  println(s"Successful kill rate ${"%.2f%%".format(
+  100 * l.successfulKillRate())} during execution of app: 
${sc.applicationId}")
+})
   sc.setJobDescription(null)
-  println("Done.")
 }
   }
 
@@ -160,8 +156,

(incubator-gluten) branch revert-5227-wip-it-log updated (1d13ab811 -> 31e12ff7b)

2024-04-01 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch revert-5227-wip-it-log
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from 1d13ab811 Revert "[VL] gluten-it: Shorten table creation and query 
runner logs (#5227)"
 add 31e12ff7b Update TpcRunner.scala

No new revisions were added by this update.

Summary of changes:
 .../src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) 01/01: Revert "[VL] gluten-it: Shorten table creation and query runner logs (#5227)"

2024-04-01 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch revert-5227-wip-it-log
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git

commit 1d13ab8115b119a2ba35fa653675c7f417dad549
Author: Hongze Zhang 
AuthorDate: Mon Apr 1 16:24:05 2024 +0800

Revert "[VL] gluten-it: Shorten table creation and query runner logs 
(#5227)"

This reverts commit f78b7b9f8c570165ef734f2718b63f2d65a8c321.
---
 .../apache/gluten/integration/tpc/TpcRunner.scala  | 19 +++
 .../scala/org/apache/spark/sql/QueryRunner.scala   | 28 ++
 .../apache/spark/sql/SparkSessionSwitcher.scala| 19 +++
 3 files changed, 28 insertions(+), 38 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
index c7ab7febd..f198a5a03 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/TpcRunner.scala
@@ -16,7 +16,8 @@
  */
 package org.apache.gluten.integration.tpc
 
-import org.apache.spark.sql.{AnalysisException, QueryRunner, RunResult, 
SparkSession}
+import org.apache.spark.sql.{QueryRunner, RunResult, SparkSession}
+
 import com.google.common.base.Preconditions
 import org.apache.commons.io.FileUtils
 
@@ -46,25 +47,21 @@ class TpcRunner(val queryResourceFolder: String, val 
dataPath: String) {
 
 object TpcRunner {
   def createTables(spark: SparkSession, dataPath: String): Unit = {
-print("Creating catalog tables: ")
-try {
-  val files = new File(dataPath).listFiles()
-  files.foreach(file => {
+val files = new File(dataPath).listFiles()
+files.foreach(
+  file => {
 if (spark.catalog.tableExists(file.getName)) {
-  print(s"${file.getName}(exists), ")
+  println("Table exists: " + file.getName)
 } else {
-  print(s"${file.getName}, ")
+  println("Creating catalog table: " + file.getName)
   spark.catalog.createTable(file.getName, file.getAbsolutePath, 
"parquet")
   try {
 spark.catalog.recoverPartitions(file.getName)
   } catch {
-case _: AnalysisException =>
+case _: Throwable =>
   }
 }
   })
-} finally {
-  println("... Done.")
-}
   }
 
   private def delete(path: String): Unit = {
diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala 
b/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala
index 20c8a0617..a4044c925 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/QueryRunner.scala
@@ -18,12 +18,7 @@ package org.apache.spark.sql
 
 import org.apache.spark.{SparkContext, Success, TaskKilled}
 import org.apache.spark.executor.ExecutorMetrics
-import org.apache.spark.scheduler.{
-  SparkListener,
-  SparkListenerExecutorMetricsUpdate,
-  SparkListenerTaskEnd,
-  SparkListenerTaskStart
-}
+import org.apache.spark.scheduler.{SparkListener, 
SparkListenerExecutorMetricsUpdate, SparkListenerTaskEnd, 
SparkListenerTaskStart}
 import org.apache.spark.sql.KillTaskListener.INIT_WAIT_TIME_MS
 
 import com.google.common.base.Preconditions
@@ -50,7 +45,8 @@ object QueryRunner {
 "ProcessTreePythonVMemory",
 "ProcessTreePythonRSSMemory",
 "ProcessTreeOtherVMemory",
-"ProcessTreeOtherRSSMemory")
+"ProcessTreeOtherRSSMemory"
+  )
 
   def runTpcQuery(
   spark: SparkSession,
@@ -80,7 +76,7 @@ object QueryRunner {
 }
 killTaskListener.foreach(sc.addSparkListener(_))
 
-print(s"Executing SQL query from resource path $queryPath... ")
+println(s"Executing SQL query from resource path $queryPath...")
 try {
   val sql = resourceToString(queryPath)
   val prev = System.nanoTime()
@@ -94,13 +90,13 @@ object QueryRunner {
   RunResult(rows, millis, collectedMetrics)
 } finally {
   sc.removeSparkListener(metricsListener)
-  killTaskListener.foreach(l => {
-sc.removeSparkListener(l)
-println(s"Successful kill rate ${"%.2f%%"
-  .format(100 * l.successfulKillRate())} during execution of app: 
${sc.applicationId}")
-  })
+  killTaskListener.foreach(
+l => {
+  sc.removeSparkListener(l)
+  println(s"Successful kill rate ${"%.2f%%".format(
+  100 * l.successfulKillRate())} during execution of app: 
${sc.applicationId}")
+})
   sc.setJobDescri

(incubator-gluten) branch revert-5227-wip-it-log created (now 1d13ab811)

2024-04-01 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch revert-5227-wip-it-log
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


  at 1d13ab811 Revert "[VL] gluten-it: Shorten table creation and query 
runner logs (#5227)"

This branch includes the following new commits:

 new 1d13ab811 Revert "[VL] gluten-it: Shorten table creation and query 
runner logs (#5227)"

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated (e03a238f5 -> f78b7b9f8)

2024-04-01 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from e03a238f5 [VL] gluten-it: Remove specific result matching code for Q65 
(#5226)
 add f78b7b9f8 [VL] gluten-it: Shorten table creation and query runner logs 
(#5227)

No new revisions were added by this update.

Summary of changes:
 .../apache/gluten/integration/tpc/TpcRunner.scala  | 19 ---
 .../scala/org/apache/spark/sql/QueryRunner.scala   | 28 --
 .../apache/spark/sql/SparkSessionSwitcher.scala| 19 ---
 3 files changed, 38 insertions(+), 28 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] gluten-it: Remove specific result matching code for Q65 (#5226)

2024-04-01 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new e03a238f5 [VL] gluten-it: Remove specific result matching code for Q65 
(#5226)
e03a238f5 is described below

commit e03a238f55103e473ef5606ab10a6208ef5fc242
Author: Hongze Zhang 
AuthorDate: Mon Apr 1 15:34:22 2024 +0800

[VL] gluten-it: Remove specific result matching code for Q65 (#5226)
---
 .../org/apache/gluten/integration/tpc/action/QueriesCompare.scala| 5 +
 1 file changed, 1 insertion(+), 4 deletions(-)

diff --git 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
index 32b7c59c8..14dcfe12c 100644
--- 
a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
+++ 
b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/tpc/action/QueriesCompare.scala
@@ -189,10 +189,7 @@ object QueriesCompare {
   val result = runner.runTpcQuery(sessionSwitcher.spark(), testDesc, id, 
explain = explain)
   val resultRows = result.rows
   val error = TestUtils.compareAnswers(resultRows, expectedRows, sort = 
true)
-  // FIXME: This is too hacky
-  // A list of query ids whose corresponding query results can differ 
because of order.
-  val unorderedQueries = Seq("q65")
-  if (error.isEmpty || unorderedQueries.contains(id)) {
+  if (error.isEmpty) {
 println(
   s"Successfully ran query $id, result check was passed. " +
 s"Returned row count: ${resultRows.length}, expected: 
${expectedRows.length}")


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated (598727793 -> 8905dc17e)

2024-03-31 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


from 598727793 [Gluten-5152][CH]Support Optimize and VACUUM command for 
clickhouse tables (#5153)
 add 8905dc17e [VL] CI: Enable GHA dependency cache on static Velox build 
(#5145)

No new revisions were added by this update.

Summary of changes:
 .github/workflows/velox_docker.yml | 11 ++-
 1 file changed, 10 insertions(+), 1 deletion(-)


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [CH] Move project transformer rewriting code to CH backend (#5171)

2024-03-29 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 20d108a5c [CH] Move project transformer rewriting code to CH backend 
(#5171)
20d108a5c is described below

commit 20d108a5ccc4e6dd2bebcfedffed448043568235
Author: Hongze Zhang 
AuthorDate: Fri Mar 29 15:47:30 2024 +0800

[CH] Move project transformer rewriting code to CH backend (#5171)
---
 .../clickhouse/CHSparkPlanExecApi.scala| 43 +
 .../backendsapi/SparkPlanExecApi.scala |  5 +++
 .../BasicPhysicalOperatorTransformer.scala | 45 +++---
 3 files changed, 54 insertions(+), 39 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
 
b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 781884ad5..eef27665f 100644
--- 
a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ 
b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -108,6 +108,49 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
 RowToCHNativeColumnarExec(child)
   }
 
+  override def genProjectExecTransformer(
+  projectList: Seq[NamedExpression],
+  child: SparkPlan): ProjectExecTransformer = {
+def processProjectExecTransformer(projectList: Seq[NamedExpression]): 
Seq[NamedExpression] = {
+  // When there is a MergeScalarSubqueries which will create the 
named_struct with the
+  // same name, looks like {'bloomFilter', BF1, 'bloomFilter', BF2}
+  // or {'count(1)', count(1)#111L, 'avg(a)', avg(a)#222L, 'count(1)', 
count(1)#333L},
+  // it will cause problem for ClickHouse backend,
+  // which cannot tolerate duplicate type names in struct type,
+  // so we need to rename 'nameExpr' in the named_struct to make them 
unique
+  // after executing the MergeScalarSubqueries.
+  var needToReplace = false
+  val newProjectList = projectList.map {
+case alias @ Alias(cns @ CreateNamedStruct(children: Seq[Expression]), 
"mergedValue") =>
+  // check whether there are some duplicate names
+  if (cns.nameExprs.distinct.size == cns.nameExprs.size) {
+alias
+  } else {
+val newChildren = children
+  .grouped(2)
+  .flatMap {
+case Seq(name: Literal, value: NamedExpression) =>
+  val newLiteral = Literal(name.toString() + "#" + 
value.exprId.id)
+  Seq(newLiteral, value)
+case Seq(name, value) => Seq(name, value)
+  }
+  .toSeq
+needToReplace = true
+Alias.apply(CreateNamedStruct(newChildren), 
"mergedValue")(alias.exprId)
+  }
+case other: NamedExpression => other
+  }
+
+  if (!needToReplace) {
+projectList
+  } else {
+newProjectList
+  }
+}
+
+
ProjectExecTransformer.createUnsafe(processProjectExecTransformer(projectList), 
child)
+  }
+
   /**
* Generate FilterExecTransformer.
*
diff --git 
a/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala
 
b/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala
index 759f7cfad..711b49645 100644
--- 
a/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala
+++ 
b/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala
@@ -85,6 +85,11 @@ trait SparkPlanExecApi {
   def genHiveTableScanExecTransformer(plan: SparkPlan): 
HiveTableScanExecTransformer =
 HiveTableScanExecTransformer(plan)
 
+  def genProjectExecTransformer(
+  projectList: Seq[NamedExpression],
+  child: SparkPlan): ProjectExecTransformer =
+ProjectExecTransformer.createUnsafe(projectList, child)
+
   /** Generate HashAggregateExecTransformer. */
   def genHashAggregateExecTransformer(
   requiredChildDistributionExpressions: Option[Seq[Expression]],
diff --git 
a/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala
 
b/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala
index 9fa252016..3a32b5eb8 100644
--- 
a/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala
+++ 
b/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala
@@ -248,47 +248,14 @@ case class ProjectExecTransformer private (projectList: 
Seq[NamedExpression], ch
 copy(child = newChild)
 }
 object ProjectExecTransformer {
-  private def processProjectExecTransformer(
-  projectList: Seq[NamedExpression]

(incubator-gluten) branch main updated: [CORE][VL] ACBO: Add GlutenMetadataModel, move Gluten schema def from property model to metadata model (#5159)

2024-03-28 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new c6639efe0 [CORE][VL] ACBO: Add GlutenMetadataModel, move Gluten schema 
def from property model to metadata model (#5159)
c6639efe0 is described below

commit c6639efe0504a2193db1e1ed49a324bc1c7b91d7
Author: Hongze Zhang 
AuthorDate: Fri Mar 29 12:55:46 2024 +0800

[CORE][VL] ACBO: Add GlutenMetadataModel, move Gluten schema def from 
property model to metadata model (#5159)
---
 .../io/glutenproject/planner/VeloxCboSuite.scala   |  15 +-
 .../src/main/scala/io/glutenproject/cbo/Cbo.scala  |  37 -
 .../scala/io/glutenproject/cbo/CboCluster.scala|  11 +-
 .../main/scala/io/glutenproject/cbo/CboGroup.scala |   2 +-
 .../cbo/{PlanModel.scala => MetadataModel.scala}   |  22 +--
 .../scala/io/glutenproject/cbo/PlanModel.scala |   2 +-
 .../glutenproject/cbo/memo/ForwardMemoTable.scala  |  34 ++--
 .../scala/io/glutenproject/cbo/memo/Memo.scala |   7 +-
 .../io/glutenproject/cbo/memo/MemoTable.scala  |   2 +-
 .../io/glutenproject/cbo/CboMetadataSuite.scala| 183 +
 .../io/glutenproject/cbo/CboOperationSuite.scala   |  28 ++--
 .../io/glutenproject/cbo/CboPropertySuite.scala|  44 +++--
 .../test/scala/io/glutenproject/cbo/CboSuite.scala |  45 +++--
 .../scala/io/glutenproject/cbo/CboSuiteBase.scala  |  27 ++-
 .../io/glutenproject/cbo/mock/MockMemoState.scala  |  14 +-
 .../io/glutenproject/cbo/path/CboPathSuite.scala   |  11 +-
 .../glutenproject/cbo/path/PathFinderSuite.scala   |  17 +-
 .../io/glutenproject/cbo/path/PathMaskSuite.scala  |   4 +-
 .../io/glutenproject/cbo/path/WizardSuite.scala|  17 +-
 .../io/glutenproject/cbo/rule/PatternSuite.scala   |  22 ++-
 .../cbo/specific/CyclicSearchSpaceSuite.scala  |  15 +-
 .../cbo/specific/DistributedSuite.scala|  27 ++-
 .../cbo/specific/JoinReorderSuite.scala|  41 +++--
 .../extension/columnar/EnumeratedTransform.scala   |   4 +-
 .../glutenproject/planner/GlutenOptimization.scala |   7 +-
 .../planner/metadata/GlutenMetadata.scala  |  27 +--
 .../planner/metadata/GlutenMetadataModel.scala |  48 ++
 .../planner/plan/GlutenPlanModel.scala |  18 +-
 .../planner/property/GlutenPropertyModel.scala |  35 +---
 29 files changed, 555 insertions(+), 211 deletions(-)

diff --git 
a/backends-velox/src/test/scala/io/glutenproject/planner/VeloxCboSuite.scala 
b/backends-velox/src/test/scala/io/glutenproject/planner/VeloxCboSuite.scala
index 83ce7c69a..2fbc1e642 100644
--- a/backends-velox/src/test/scala/io/glutenproject/planner/VeloxCboSuite.scala
+++ b/backends-velox/src/test/scala/io/glutenproject/planner/VeloxCboSuite.scala
@@ -16,11 +16,12 @@
  */
 package io.glutenproject.planner
 
-import io.glutenproject.cbo.{Cbo, CboSuiteBase}
+import io.glutenproject.cbo.Cbo
+import io.glutenproject.cbo.CboSuiteBase._
 import io.glutenproject.cbo.path.CboPath
 import io.glutenproject.cbo.property.PropertySet
 import io.glutenproject.cbo.rule.{CboRule, Shape, Shapes}
-import io.glutenproject.planner.property.GlutenProperties.{Conventions, 
Schemas}
+import io.glutenproject.planner.property.GlutenProperties.Conventions
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
@@ -41,7 +42,7 @@ class VeloxCboSuite extends SharedSparkSession {
   test("C2R, R2C - explicitly requires any properties") {
 val in = RowUnary(RowLeaf())
 val planner =
-  newCbo().newPlanner(in, PropertySet(List(Conventions.ANY, Schemas.ANY)))
+  newCbo().newPlanner(in, PropertySet(List(Conventions.ANY)))
 val out = planner.plan()
 assert(out == RowUnary(RowLeaf()))
   }
@@ -49,7 +50,7 @@ class VeloxCboSuite extends SharedSparkSession {
   test("C2R, R2C - requires columnar output") {
 val in = RowUnary(RowLeaf())
 val planner =
-  newCbo().newPlanner(in, PropertySet(List(Conventions.VANILLA_COLUMNAR, 
Schemas.ANY)))
+  newCbo().newPlanner(in, PropertySet(List(Conventions.VANILLA_COLUMNAR)))
 val out = planner.plan()
 assert(out == RowToColumnarExec(RowUnary(RowLeaf(
   }
@@ -58,7 +59,7 @@ class VeloxCboSuite extends SharedSparkSession {
 val in =
   
ColumnarUnary(RowUnary(RowUnary(ColumnarUnary(RowUnary(RowUnary(ColumnarUnary(RowLeaf(
 val planner =
-  newCbo().newPlanner(in, PropertySet(List(Conventions.ROW_BASED, 
Schemas.ANY)))
+  newCbo().newPlanner(in, PropertySet(List(Conventions.ROW_BASED)))
 val out = planner.plan()
 assert(out == ColumnarToRowExec(ColumnarUnary(
   
RowToColumnarExec(RowUnary(RowUnary(ColumnarToRowExec(ColumnarUnary(RowToColumnarExec(
@@ -82,7 +83,7 @@ class VeloxCboSuite extends SharedSparkSession {
   
ColumnarUnary(RowUnary(RowUnary(ColumnarUnary(RowUnary(RowUnar

(incubator-gluten) branch main updated: [GLUTEN-5136][VL] Duplicated output from Spark-to-Velox broadcast relation conversion (#5141)

2024-03-26 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new e4fe9baec [GLUTEN-5136][VL] Duplicated output from Spark-to-Velox 
broadcast relation conversion (#5141)
e4fe9baec is described below

commit e4fe9baeccde07e2938d5f186151c43591e91720
Author: Hongze Zhang 
AuthorDate: Wed Mar 27 12:54:29 2024 +0800

[GLUTEN-5136][VL] Duplicated output from Spark-to-Velox broadcast relation 
conversion (#5141)
---
 .../apache/spark/sql/execution/BroadcastUtils.scala| 18 +++---
 1 file changed, 15 insertions(+), 3 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala
 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala
index a0f28c5ab..ad7694ea2 100644
--- 
a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala
+++ 
b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala
@@ -26,7 +26,7 @@ import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, 
BroadcastPartitioning, IdentityBroadcastMode, Partitioning}
-import org.apache.spark.sql.execution.joins.{HashedRelation, 
HashedRelationBroadcastMode}
+import org.apache.spark.sql.execution.joins.{HashedRelation, 
HashedRelationBroadcastMode, LongHashedRelation}
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.vectorized.ColumnarBatch
 import org.apache.spark.util.TaskResources
@@ -96,9 +96,8 @@ object BroadcastUtils {
 // HashedRelation to ColumnarBuildSideRelation.
 val fromBroadcast = from.asInstanceOf[Broadcast[HashedRelation]]
 val fromRelation = fromBroadcast.value.asReadOnlyCopy()
-val keys = fromRelation.keys()
 val toRelation = TaskResources.runUnsafe {
-  val batchItr: Iterator[ColumnarBatch] = fn(keys.flatMap(key => 
fromRelation.get(key)))
+  val batchItr: Iterator[ColumnarBatch] = 
fn(reconstructRows(fromRelation))
   val serialized: Array[Array[Byte]] = serializeStream(batchItr) match 
{
 case ColumnarBatchSerializeResult.EMPTY =>
   Array()
@@ -170,4 +169,17 @@ object BroadcastUtils {
   }
 serializeResult
   }
+
+  private def reconstructRows(relation: HashedRelation): Iterator[InternalRow] 
= {
+// It seems that LongHashedRelation and UnsafeHashedRelation don't follow 
the same
+//  criteria while getting values from them.
+// Should review the internals of this part of code.
+relation match {
+  case relation: LongHashedRelation if relation.keyIsUnique =>
+relation.keys().map(k => relation.getValue(k))
+  case relation: LongHashedRelation if !relation.keyIsUnique =>
+relation.keys().flatMap(k => relation.get(k))
+  case other => other.valuesWithKeyIndex().map(_.getValue)
+}
+  }
 }


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Velox patch to avoid installing libunwind-dev no longer works (#5127)

2024-03-26 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 2aa60d0ea [VL] Velox patch to avoid installing libunwind-dev no longer 
works (#5127)
2aa60d0ea is described below

commit 2aa60d0eae8fdd0f4020842c5233ca8a3197bd5e
Author: Hongze Zhang 
AuthorDate: Wed Mar 27 08:26:33 2024 +0800

[VL] Velox patch to avoid installing libunwind-dev no longer works (#5127)
---
 ep/build-velox/src/get_velox.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh
index 767585e91..26e7a9cd0 100755
--- a/ep/build-velox/src/get_velox.sh
+++ b/ep/build-velox/src/get_velox.sh
@@ -86,7 +86,7 @@ function process_setup_ubuntu {
   # need set BUILD_SHARED_LIBS flag for thrift
   sed -i  "/facebook\/fbthrift/{n;s/cmake_install 
-DBUILD_TESTS=OFF/cmake_install -DBUILD_TESTS=OFF -DBUILD_SHARED_LIBS=OFF/;}" 
scripts/setup-ubuntu.sh
   # Do not install libunwind which can cause interruption when catching native 
exception.
-  sed -i 's/sudo --preserve-env apt install -y libunwind-dev && //' 
scripts/setup-ubuntu.sh
+  sed -i 's/${SUDO} apt install -y libunwind-dev//' scripts/setup-ubuntu.sh
   sed -i '/ccache/a\  *thrift* \\' scripts/setup-ubuntu.sh
   sed -i '/ccache/a\  libiberty-dev \\' scripts/setup-ubuntu.sh
   sed -i '/ccache/a\  libxml2-dev \\' scripts/setup-ubuntu.sh


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Fix compression type 2 issue (#5121)

2024-03-25 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new fdd5ca2e1 [VL] Fix compression type 2 issue (#5121)
fdd5ca2e1 is described below

commit fdd5ca2e14af299b50e5a80aeb0e48a65dd49057
Author: Hongze Zhang 
AuthorDate: Tue Mar 26 13:57:08 2024 +0800

[VL] Fix compression type 2 issue (#5121)
---
 dev/vcpkg/ports/folly/fix-deps.patch | 15 ---
 1 file changed, 8 insertions(+), 7 deletions(-)

diff --git a/dev/vcpkg/ports/folly/fix-deps.patch 
b/dev/vcpkg/ports/folly/fix-deps.patch
index 20cd752d5..179fa71be 100644
--- a/dev/vcpkg/ports/folly/fix-deps.patch
+++ b/dev/vcpkg/ports/folly/fix-deps.patch
@@ -1,8 +1,8 @@
 diff --git a/CMake/folly-config.cmake.in b/CMake/folly-config.cmake.in
-index 1689f9a..e5d3e22 100644
+index 0b96f0a10..c90110287 100644
 --- a/CMake/folly-config.cmake.in
 +++ b/CMake/folly-config.cmake.in
-@@ -28,10 +28,35 @@ endif()
+@@ -29,10 +29,35 @@ endif()
  set(FOLLY_LIBRARIES Folly::folly)
  
  # Find folly's dependencies
@@ -41,7 +41,7 @@ index 1689f9a..e5d3e22 100644
  context
  filesystem
 diff --git a/CMake/folly-deps.cmake b/CMake/folly-deps.cmake
-index 4b78e9f..ac83c99 100644
+index 4b78e9f02..eb77e29c9 100644
 --- a/CMake/folly-deps.cmake
 +++ b/CMake/folly-deps.cmake
 @@ -35,7 +35,7 @@ else()
@@ -116,7 +116,7 @@ index 4b78e9f..ac83c99 100644
  endif()
  
  find_package(OpenSSL 1.1.1 MODULE REQUIRED)
-@@ -104,25 +105,29 @@ if (LIBLZMA_FOUND)
+@@ -104,25 +105,30 @@ if (LIBLZMA_FOUND)
list(APPEND FOLLY_LINK_LIBRARIES ${LIBLZMA_LIBRARIES})
  endif()
  
@@ -128,6 +128,7 @@ index 4b78e9f..ac83c99 100644
 +if (NOT CMAKE_DISABLE_FIND_PACKAGE_LZ4)
 +  find_package(lz4 CONFIG)
 +  if(TARGET lz4::lz4)
++set(FOLLY_HAVE_LIBLZ4 1)
 +list(APPEND FOLLY_LINK_LIBRARIES lz4::lz4)
 +  endif()
  endif()
@@ -161,7 +162,7 @@ index 4b78e9f..ac83c99 100644
  endif()
  
  find_package(LibDwarf)
-@@ -137,13 +142,18 @@ find_package(LibAIO)
+@@ -137,13 +143,18 @@ find_package(LibAIO)
  list(APPEND FOLLY_LINK_LIBRARIES ${LIBAIO_LIBRARIES})
  list(APPEND FOLLY_INCLUDE_DIRECTORIES ${LIBAIO_INCLUDE_DIRS})
  
@@ -183,7 +184,7 @@ index 4b78e9f..ac83c99 100644
  
  list(APPEND FOLLY_LINK_LIBRARIES ${CMAKE_DL_LIBS})
  list(APPEND CMAKE_REQUIRED_LIBRARIES ${CMAKE_DL_LIBS})
-@@ -154,9 +164,9 @@ if (PYTHON_EXTENSIONS)
+@@ -154,9 +165,9 @@ if (PYTHON_EXTENSIONS)
  endif ()
  
  find_package(LibUnwind)
@@ -195,7 +196,7 @@ index 4b78e9f..ac83c99 100644
set(FOLLY_HAVE_LIBUNWIND ON)
  endif()
  if (CMAKE_SYSTEM_NAME MATCHES "FreeBSD")
-@@ -299,11 +309,7 @@ endif()
+@@ -299,11 +310,7 @@ endif()
  
  add_library(folly_deps INTERFACE)
  


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Remove docs about threshold-based spill

2024-03-21 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 201f322b0 [VL] Remove docs about threshold-based spill
201f322b0 is described below

commit 201f322b0e8495e277237672f286828d683600a4
Author: Hongze Zhang 
AuthorDate: Fri Mar 22 10:42:15 2024 +0800

[VL] Remove docs about threshold-based spill
---
 docs/get-started/Velox.md | 3 ---
 1 file changed, 3 deletions(-)

diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md
index 8a9001310..acaf9f798 100644
--- a/docs/get-started/Velox.md
+++ b/docs/get-started/Velox.md
@@ -345,9 +345,6 @@ Using the following configuration options to customize 
spilling:
 | spark.gluten.sql.columnar.backend.velox.aggregationSpillEnabled  | 
true  | Whether spill is enabled on aggregations

  |
 | spark.gluten.sql.columnar.backend.velox.joinSpillEnabled | 
true  | Whether spill is enabled on joins   

  |
 | spark.gluten.sql.columnar.backend.velox.orderBySpillEnabled  | 
true  | Whether spill is enabled on sorts   

  |
-| spark.gluten.sql.columnar.backend.velox.aggregationSpillMemoryThreshold  | 0 
| Memory limit before spilling to disk for aggregations, per Spark 
task. Unit: byte
 |
-| spark.gluten.sql.columnar.backend.velox.joinSpillMemoryThreshold | 0 
| Memory limit before spilling to disk for joins, per Spark task. 
Unit: byte  
  |
-| spark.gluten.sql.columnar.backend.velox.orderBySpillMemoryThreshold  | 0 
| Memory limit before spilling to disk for sorts, per Spark task. 
Unit: byte  
  |
 | spark.gluten.sql.columnar.backend.velox.maxSpillLevel| 4 
| The max allowed spilling level with zero being the initial 
spilling level  
   |
 | spark.gluten.sql.columnar.backend.velox.maxSpillFileSize | 
20MB  | The max allowed spill file size. If it is zero, then there is 
no limit
|
 | spark.gluten.sql.columnar.backend.velox.minSpillRunSize  | 
268435456 | The min spill run size limit used to select partitions for 
spilling
   |


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [DOC] Remove arrow version setting with 11.0.0-gluten

2024-03-21 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 4e732d53f [DOC] Remove arrow version setting with 11.0.0-gluten
4e732d53f is described below

commit 4e732d53f94bcf9b00eea7b1f88472e4863153d8
Author: PHILO-HE 
AuthorDate: Thu Mar 21 16:54:56 2024 +0800

[DOC] Remove arrow version setting with 11.0.0-gluten
---
 docs/developers/HowTo.md   | 2 +-
 docs/developers/MicroBenchmarks.md | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/developers/HowTo.md b/docs/developers/HowTo.md
index a656cd52e..338b49b98 100644
--- a/docs/developers/HowTo.md
+++ b/docs/developers/HowTo.md
@@ -57,7 +57,7 @@ cd ${GLUTEN_HOME}
 mvn clean package -Pspark-3.2 -Pbackends-velox -Prss
 mvn test -Pspark-3.2 -Pbackends-velox -Prss -pl backends-velox \
 -am -DtagsToInclude="io.glutenproject.tags.GenerateExample" \
--Dtest=none -DfailIfNoTests=false -Darrow.version=11.0.0-gluten \
+-Dtest=none -DfailIfNoTests=false \
 -Dexec.skip
 ```
 
diff --git a/docs/developers/MicroBenchmarks.md 
b/docs/developers/MicroBenchmarks.md
index 5ee017909..7a18430fe 100644
--- a/docs/developers/MicroBenchmarks.md
+++ b/docs/developers/MicroBenchmarks.md
@@ -36,7 +36,7 @@ cd /path/to/gluten/
 
 # Run test to generate input data files. If you are using spark 3.3, replace 
-Pspark-3.2 with -Pspark-3.3
 mvn test -Pspark-3.2 -Pbackends-velox -Prss -pl backends-velox -am \
--DtagsToInclude="io.glutenproject.tags.GenerateExample" -Dtest=none 
-DfailIfNoTests=false -Darrow.version=11.0.0-gluten -Dexec.skip
+-DtagsToInclude="io.glutenproject.tags.GenerateExample" -Dtest=none 
-DfailIfNoTests=false -Dexec.skip
 ```
 
 The generated example files are placed in gluten/backends-velox:


-
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org



(incubator-gluten) branch main updated: [VL] Untangle code of TransformPreOverrides (#4888)

2024-03-14 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 52c0a2978 [VL] Untangle code of TransformPreOverrides (#4888)
52c0a2978 is described below

commit 52c0a29785582a446b71dbf20f5271a865fec2cf
Author: Hongze Zhang 
AuthorDate: Thu Mar 14 16:43:00 2024 +0800

[VL] Untangle code of TransformPreOverrides (#4888)
---
 .../clickhouse/CHSparkPlanExecApi.scala|   8 +-
 .../extension/columnar/MiscColumnarRules.scala | 789 +++--
 2 files changed, 428 insertions(+), 369 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
 
b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 17a9e8d67..aca06cf88 100644
--- 
a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ 
b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -197,7 +197,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
 if (projectExpressions.isEmpty) {
   return (0, plan.outputPartitioning, plan.child)
 }
-val project = TransformPreOverrides().replaceWithTransformerPlan(
+// FIXME: The operation happens inside ReplaceSingleNode().
+//  Caller may not know it adds project on top of the shuffle.
+val project = TransformPreOverrides().apply(
   AddTransformHintRule().apply(
 ProjectExec(plan.child.output ++ projectExpressions, plan.child)))
 var newExprs = Seq[Expression]()
@@ -220,7 +222,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
 if (projectExpressions.isEmpty) {
   return (0, plan.outputPartitioning, plan.child)
 }
-val project = TransformPreOverrides().replaceWithTransformerPlan(
+// FIXME: The operation happens inside ReplaceSingleNode().
+//  Caller may not know it adds project on top of the shuffle.
+val project = TransformPreOverrides().apply(
   AddTransformHintRule().apply(
 ProjectExec(plan.child.output ++ projectExpressions, plan.child)))
 var newOrderings = Seq[SortOrder]()
diff --git 
a/gluten-core/src/main/scala/io/glutenproject/extension/columnar/MiscColumnarRules.scala
 
b/gluten-core/src/main/scala/io/glutenproject/extension/columnar/MiscColumnarRules.scala
index 5d740153a..79fd37f3f 100644
--- 
a/gluten-core/src/main/scala/io/glutenproject/extension/columnar/MiscColumnarRules.scala
+++ 
b/gluten-core/src/main/scala/io/glutenproject/extension/columnar/MiscColumnarRules.scala
@@ -25,6 +25,7 @@ import io.glutenproject.sql.shims.SparkShimLoader
 import io.glutenproject.utils.{LogLevelUtil, PlanUtil}
 
 import org.apache.spark.api.python.EvalPythonExecTransformer
+import org.apache.spark.internal.Logging
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, 
BuildSide}
@@ -42,406 +43,461 @@ import org.apache.spark.sql.execution.window.WindowExec
 import org.apache.spark.sql.hive.HiveTableScanExecTransformer
 
 object MiscColumnarRules {
-  // This rule will conduct the conversion from Spark plan to the plan 
transformer.
-  case class TransformPreOverrides() extends Rule[SparkPlan] with LogLevelUtil 
{
-val columnarConf: GlutenConfig = GlutenConfig.getConf
-@transient private val planChangeLogger = new PlanChangeLogger[SparkPlan]()
+  object TransformPreOverrides {
+// Sub-rules of TransformPreOverrides.
 
-/**
- * Generate a plan for hash aggregation.
- * @param plan:
- *   the original Spark plan.
- * @return
- *   the actually used plan for execution.
- */
-private def genHashAggregateExec(plan: HashAggregateExec): SparkPlan = {
-  val newChild = replaceWithTransformerPlan(plan.child)
-  def transformHashAggregate(): GlutenPlan = {
-BackendsApiManager.getSparkPlanExecApiInstance
-  .genHashAggregateExecTransformer(
-plan.requiredChildDistributionExpressions,
-plan.groupingExpressions,
-plan.aggregateExpressions,
-plan.aggregateAttributes,
-plan.initialInputBufferOffset,
-plan.resultExpressions,
-newChild
-  )
+// Aggregation transformation.
+private case class AggregationTransformRule() extends Rule[SparkPlan] with 
LogLevelUtil {
+  override def apply(plan: SparkPlan): SparkPlan = plan.transformUp {
+case agg: HashAggregateExec =>
+  genHashAggregateExec(agg)
   }
 
-  // If child's output is empty, fallback or offload both the child and 
aggregat

(incubator-gluten) branch main updated: [CORE] Prior to #4893, add vanilla Spark's original scan source code to keep git history

2024-03-13 Thread hongze
This is an automated email from the ASF dual-hosted git repository.

hongze pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
 new 5644fc226 [CORE] Prior to #4893, add vanilla Spark's original scan 
source code to keep git history
5644fc226 is described below

commit 5644fc2268e56d2e629177b62808d4c3190abc61
Author: Hongze Zhang 
AuthorDate: Wed Mar 13 16:44:48 2024 +0800

[CORE] Prior to #4893, add vanilla Spark's original scan source code to 
keep git history
---
 .../sql/execution/AbstractFileSourceScanExec.scala | 524 +++
 .../datasources/v2/AbstractBatchScanExec.scala | 111 
 .../sql/execution/AbstractFileSourceScanExec.scala | 569 +
 .../datasources/v2/AbstractBatchScanExec.scala | 141 +
 .../sql/execution/AbstractFileSourceScanExec.scala | 285 +++
 .../datasources/v2/AbstractBatchScanExec.scala | 251 +
 .../sql/execution/AbstractFileSourceScanExec.scala | 273 ++
 .../datasources/v2/AbstractBatchScanExec.scala | 273 ++
 8 files changed, 2427 insertions(+)

diff --git 
a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala
 
b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala
new file mode 100644
index 0..c5d17045b
--- /dev/null
+++ 
b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala
@@ -0,0 +1,524 @@
+package org.apache.spark.sql.execution
+
+import org.apache.hadoop.fs.Path
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.catalog.BucketSpec
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, 
Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat 
=> ParquetSource}
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.util.Utils
+import org.apache.spark.util.collection.BitSet
+
+import java.util.concurrent.TimeUnit._
+import scala.collection.mutable.HashMap
+/**
+ * Physical plan node for scanning data from HadoopFsRelations.
+ *
+ * @param relation The file-based relation to scan.
+ * @param output Output attributes of the scan, including data attributes and 
partition attributes.
+ * @param requiredSchema Required schema of the underlying relation, excluding 
partition columns.
+ * @param partitionFilters Predicates to use for partition pruning.
+ * @param optionalBucketSet Bucket ids for bucket pruning.
+ * @param optionalNumCoalescedBuckets Number of coalesced buckets.
+ * @param dataFilters Filters on non-partition columns.
+ * @param tableIdentifier Identifier for the table in the metastore.
+ * @param disableBucketedScan Disable bucketed scan based on physical query 
plan, see rule
+ *[[DisableUnnecessaryBucketedScan]] for details.
+ */
+case class AbstractFileSourceScanExec(
+   @transient relation: HadoopFsRelation,
+   output: Seq[Attribute],
+   requiredSchema: StructType,
+   partitionFilters: Seq[Expression],
+   optionalBucketSet: Option[BitSet],
+   optionalNumCoalescedBuckets: Option[Int],
+   dataFilters: Seq[Expression],
+   tableIdentifier: Option[TableIdentifier],
+   disableBucketedScan: Boolean = false)
+  extends DataSourceScanExec {
+
+  // Note that some vals referring the file-based relation are lazy 
intentionally
+  // so that this plan can be canonicalized on executor side too. See 
SPARK-23731.
+  override lazy val supportsColumnar: Boolean = {
+relation.fileFormat.supportBatch(relation.sparkSession, schema)
+  }
+
+  private lazy val needsUnsafeRowConversion: Boolean = {
+if (relation.fileFormat.isInstanceOf[ParquetSource]) {
+  conf.parquetVectorizedReaderEnabled
+} else {
+  false
+}
+  }
+
+  override def vectorTypes: Option[Seq[String]] =
+relation.fileFormat.vectorTypes(
+  requiredSchema = requiredSchema,
+  partitionSchema = relation.partitionSchema,
+  relation.sparkSession.sessionState.conf)
+
+  private lazy val driverMetrics: HashMap[String, Long] = HashMap.empty
+
+  /**
+   * Send the driver-side metrics. Before calling this function, 
selectedPartitions has
+   * been initialized.