Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-19 Thread via GitHub


zhztheplayer commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1571919125


##
.github/workflows/velox_docker.yml:
##
@@ -326,6 +327,66 @@ jobs:
 --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  --random-kill-tasks
 
+  run-tpc-test-centos8-uniffle:
+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 git
+  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 clean install -P${{ matrix.spark }} -Pbackends-velox 
-Prss-uniffle -DskipTests
+  - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle 
0.8.0
+run: |
+  export MAVEN_HOME=/usr/lib/maven && \
+  export PATH=${PATH}:${MAVEN_HOME}/bin && \
+  export export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \
+  cd /opt && \
+  git clone -b branch-0.8 
https://github.com/apache/incubator-uniffle.git && \
+  cd incubator-uniffle && \
+  sed -i '250d' 
./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \
+  sed -i '228d' 
./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \
+  sed -i '226d' 
./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \
+  mvn clean install -Phadoop2.8 -DskipTests
+  cd /opt && \
+  wget -nv 
https://archive.apache.org/dist/incubator/uniffle/0.8.0/apache-uniffle-0.8.0-incubating-bin.tar.gz
 && \
+  tar xzf apache-uniffle-0.8.0-incubating-bin.tar.gz -C /opt/ && mv 
/opt/rss-0.8.0-hadoop2.8 /opt/uniffle && \
+  wget -nv 
https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz 
&& \
+  tar xzf hadoop-2.8.5.tar.gz -C /opt/
+  rm -f /opt/uniffle/jars/server/shuffle-server-0.8.0-SNAPSHOT.jar

Review Comment:
   Did you want to remove `shuffle-server-0.8.0.jar` instead of 
`shuffle-server-0.8.0-SNAPSHOT.jar`, was this a typo or something?



##
.github/workflows/velox_docker.yml:
##
@@ -326,6 +327,66 @@ jobs:
 --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  --random-kill-tasks
 
+  run-tpc-test-centos8-uniffle:
+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 git
+  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 clean install -P${{ matrix.spark }} -Pbackends-velox 
-Prss-uniffle -DskipTests
+  - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle 
0.8.0
+run: |
+  export MAVEN_HOME=/usr/lib/maven && \
+  export 

Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-11 Thread via GitHub


jackylee-ch merged PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-10 Thread via GitHub


jackylee-ch commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2048892669

   This pr is ready to be merged on my side. Would leave some time to see if 
there are any more suggestions.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-10 Thread via GitHub


zhztheplayer commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2048860082

   > @zhztheplayer, do you have any other comment?
   
   I'm good if we have tests covering the feature. Please proceed if it's ready 
to merge.
   
   Thanks @summaryzb for this work!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-10 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1559370666


##
gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java:
##
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.shuffle.writer;
+
+import org.apache.gluten.GlutenConfig;
+import org.apache.gluten.columnarbatch.ColumnarBatches;
+import org.apache.gluten.memory.memtarget.MemoryTarget;
+import org.apache.gluten.memory.memtarget.Spiller;
+import org.apache.gluten.memory.memtarget.Spillers;
+import org.apache.gluten.memory.nmm.NativeMemoryManagers;
+import org.apache.gluten.vectorized.ShuffleWriterJniWrapper;
+import org.apache.gluten.vectorized.SplitResult;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.TaskContext;
+import org.apache.spark.executor.ShuffleWriteMetrics;
+import org.apache.spark.memory.SparkMemoryUtil;
+import org.apache.spark.scheduler.MapStatus;
+import org.apache.spark.shuffle.ColumnarShuffleDependency;
+import org.apache.spark.shuffle.GlutenShuffleUtils;
+import org.apache.spark.shuffle.RssShuffleHandle;
+import org.apache.spark.shuffle.RssShuffleManager;
+import org.apache.spark.shuffle.RssSparkConfig;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.apache.spark.util.SparkResourceUtil;
+import org.apache.uniffle.client.api.ShuffleWriteClient;
+import org.apache.uniffle.common.ShuffleBlockInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+import scala.Option;
+import scala.Product2;
+import scala.collection.Iterator;
+
+public class VeloxUniffleColumnarShuffleWriter extends 
RssShuffleWriter {
+
+  private static final Logger LOG =
+  LoggerFactory.getLogger(VeloxUniffleColumnarShuffleWriter.class);
+
+  private long nativeShuffleWriter = -1L;
+
+  private boolean stopping = false;
+  private int compressThreshold = 
GlutenConfig.getConf().columnarShuffleCompressionThreshold();
+  private double reallocThreshold = 
GlutenConfig.getConf().columnarShuffleReallocThreshold();
+  private String compressionCodec;
+  private int compressionLevel;
+  private int partitionId;
+
+  private ShuffleWriterJniWrapper jniWrapper = 
ShuffleWriterJniWrapper.create();
+  private SplitResult splitResult;
+  private int nativeBufferSize = GlutenConfig.getConf().maxBatchSize();
+  private int bufferSize;
+  private PartitionPusher partitionPusher;
+
+  private final ColumnarShuffleDependency columnarDep;
+  private final SparkConf sparkConf;
+
+  private long availableOffHeapPerTask() {
+return SparkMemoryUtil.getCurrentAvailableOffHeapMemory()
+/ SparkResourceUtil.getTaskSlots(sparkConf);
+  }
+
+  public VeloxUniffleColumnarShuffleWriter(
+  int partitionId,
+  String appId,
+  int shuffleId,
+  String taskId,
+  long taskAttemptId,
+  ShuffleWriteMetrics shuffleWriteMetrics,
+  RssShuffleManager shuffleManager,
+  SparkConf sparkConf,
+  ShuffleWriteClient shuffleWriteClient,
+  RssShuffleHandle rssHandle,
+  Function taskFailureCallback,
+  TaskContext context) {
+super(
+appId,
+shuffleId,
+taskId,
+taskAttemptId,
+shuffleWriteMetrics,
+shuffleManager,
+sparkConf,
+shuffleWriteClient,
+rssHandle,
+taskFailureCallback,
+context);
+columnarDep = (ColumnarShuffleDependency) 
rssHandle.getDependency();
+this.partitionId = partitionId;
+this.sparkConf = sparkConf;
+bufferSize =
+(int)
+sparkConf.getSizeAsBytes(
+RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key(),
+RssSparkConfig.RSS_WRITER_BUFFER_SIZE.defaultValue().get());
+compressionCodec = GlutenShuffleUtils.getCompressionCodec(sparkConf);

Review Comment:
   Fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: 

Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-10 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2047447805

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


PHILO-HE commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1558784873


##
gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java:
##
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.shuffle.writer;
+
+import org.apache.gluten.GlutenConfig;
+import org.apache.gluten.columnarbatch.ColumnarBatches;
+import org.apache.gluten.memory.memtarget.MemoryTarget;
+import org.apache.gluten.memory.memtarget.Spiller;
+import org.apache.gluten.memory.memtarget.Spillers;
+import org.apache.gluten.memory.nmm.NativeMemoryManagers;
+import org.apache.gluten.vectorized.ShuffleWriterJniWrapper;
+import org.apache.gluten.vectorized.SplitResult;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.TaskContext;
+import org.apache.spark.executor.ShuffleWriteMetrics;
+import org.apache.spark.memory.SparkMemoryUtil;
+import org.apache.spark.scheduler.MapStatus;
+import org.apache.spark.shuffle.ColumnarShuffleDependency;
+import org.apache.spark.shuffle.GlutenShuffleUtils;
+import org.apache.spark.shuffle.RssShuffleHandle;
+import org.apache.spark.shuffle.RssShuffleManager;
+import org.apache.spark.shuffle.RssSparkConfig;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.apache.spark.util.SparkResourceUtil;
+import org.apache.uniffle.client.api.ShuffleWriteClient;
+import org.apache.uniffle.common.ShuffleBlockInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+import scala.Option;
+import scala.Product2;
+import scala.collection.Iterator;
+
+public class VeloxUniffleColumnarShuffleWriter extends 
RssShuffleWriter {
+
+  private static final Logger LOG =
+  LoggerFactory.getLogger(VeloxUniffleColumnarShuffleWriter.class);
+
+  private long nativeShuffleWriter = -1L;
+
+  private boolean stopping = false;
+  private int compressThreshold = 
GlutenConfig.getConf().columnarShuffleCompressionThreshold();
+  private double reallocThreshold = 
GlutenConfig.getConf().columnarShuffleReallocThreshold();
+  private String compressionCodec;
+  private int compressionLevel;
+  private int partitionId;
+
+  private ShuffleWriterJniWrapper jniWrapper = 
ShuffleWriterJniWrapper.create();
+  private SplitResult splitResult;
+  private int nativeBufferSize = GlutenConfig.getConf().maxBatchSize();
+  private int bufferSize;
+  private PartitionPusher partitionPusher;
+
+  private final ColumnarShuffleDependency columnarDep;
+  private final SparkConf sparkConf;
+
+  private long availableOffHeapPerTask() {
+return SparkMemoryUtil.getCurrentAvailableOffHeapMemory()
+/ SparkResourceUtil.getTaskSlots(sparkConf);
+  }
+
+  public VeloxUniffleColumnarShuffleWriter(
+  int partitionId,
+  String appId,
+  int shuffleId,
+  String taskId,
+  long taskAttemptId,
+  ShuffleWriteMetrics shuffleWriteMetrics,
+  RssShuffleManager shuffleManager,
+  SparkConf sparkConf,
+  ShuffleWriteClient shuffleWriteClient,
+  RssShuffleHandle rssHandle,
+  Function taskFailureCallback,
+  TaskContext context) {
+super(
+appId,
+shuffleId,
+taskId,
+taskAttemptId,
+shuffleWriteMetrics,
+shuffleManager,
+sparkConf,
+shuffleWriteClient,
+rssHandle,
+taskFailureCallback,
+context);
+columnarDep = (ColumnarShuffleDependency) 
rssHandle.getDependency();
+this.partitionId = partitionId;
+this.sparkConf = sparkConf;
+bufferSize =
+(int)
+sparkConf.getSizeAsBytes(
+RssSparkConfig.RSS_WRITER_BUFFER_SIZE.key(),
+RssSparkConfig.RSS_WRITER_BUFFER_SIZE.defaultValue().get());
+compressionCodec = GlutenShuffleUtils.getCompressionCodec(sparkConf);

Review Comment:
   I note celeborn honors Spark's config for SHUFFLE_COMPRESS. Should uniffle 
do the same check? See 
https://github.com/apache/incubator-gluten/commit/e182d659b78ef7e384a2020c903dbf65c502012d.



-- 

Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1557317912


##
pom.xml:
##
@@ -237,6 +238,15 @@
 gluten-celeborn
   
 
+
+  rss-uniffle

Review Comment:
   All the naming issue between celeborn an uniffle will be resolved in  
https://github.com/apache/incubator-gluten/issues/5335



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1557316749


##
cpp/core/jni/JniWrapper.cc:
##
@@ -915,6 +915,23 @@ JNIEXPORT jlong JNICALL 
Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe
 std::move(partitionWriterOptions),
 memoryManager->getArrowMemoryPool(),
 std::move(celebornClient));
+  } else if (partitionWriterType == "uniffle") {
+jclass unifflePartitionPusherClass =
+createGlobalClassReferenceOrError(env, 
"Lorg/apache/spark/shuffle/writer/PartitionPusher;");
+jmethodID unifflePushPartitionDataMethod =
+getMethodIdOrError(env, unifflePartitionPusherClass, 
"pushPartitionData", "(I[BI)I");
+JavaVM* vm;
+if (env->GetJavaVM() != JNI_OK) {
+  throw gluten::GlutenException("Unable to get JavaVM instance");
+}
+// rename CelebornClient RssClient
+std::shared_ptr uniffleClient =
+std::make_shared(vm, partitionPusher, 
unifflePushPartitionDataMethod);
+partitionWriter = std::make_unique(

Review Comment:
   Follow this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1557317912


##
pom.xml:
##
@@ -237,6 +238,15 @@
 gluten-celeborn
   
 
+
+  rss-uniffle

Review Comment:
   Resolve it in the issue 
https://github.com/apache/incubator-gluten/issues/5335



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


jackylee-ch commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1557306763


##
pom.xml:
##
@@ -237,6 +238,15 @@
 gluten-celeborn
   
 
+
+  rss-uniffle

Review Comment:
   nit: It is better to rename the rss profile to celeborn and rename 
rss-uniffle to uniffle to speicify the build module.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1557194618


##
cpp/core/jni/JniWrapper.cc:
##
@@ -888,6 +888,23 @@ JNIEXPORT jlong JNICALL 
Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper
 std::move(partitionWriterOptions),
 memoryManager->getArrowMemoryPool(),
 std::move(celebornClient));
+  } else if (partitionWriterType == "uniffle") {
+jclass unifflePartitionPusherClass =
+createGlobalClassReferenceOrError(env, 
"Lorg/apache/spark/shuffle/writer/PartitionPusher;");
+jmethodID unifflePushPartitionDataMethod =
+getMethodIdOrError(env, unifflePartitionPusherClass, 
"pushPartitionData", "(I[B)I");
+JavaVM* vm;
+if (env->GetJavaVM() != JNI_OK) {
+  throw gluten::GlutenException("Unable to get JavaVM instance");
+}
+// rename CelebornClient RssClient
+std::shared_ptr celebornClient =
+std::make_shared(vm, partitionPusher, 
unifflePushPartitionDataMethod);

Review Comment:
   fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


summaryzb commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2044426642

   @PHILO-HE PTAL


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-09 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1557186609


##
gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/gluten/uniffle/GlutenRssShuffleManager.java:
##
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.shuffle.gluten.uniffle;
+
+import org.apache.spark.ShuffleDependency;
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkEnv;
+import org.apache.spark.TaskContext;
+import org.apache.spark.executor.ShuffleWriteMetrics;
+import org.apache.spark.shuffle.ColumnarShuffleDependency;
+import org.apache.spark.shuffle.RssShuffleHandle;
+import org.apache.spark.shuffle.RssShuffleManager;
+import org.apache.spark.shuffle.RssSparkConfig;
+import org.apache.spark.shuffle.ShuffleHandle;
+import org.apache.spark.shuffle.ShuffleWriteMetricsReporter;
+import org.apache.spark.shuffle.ShuffleWriter;
+import org.apache.spark.shuffle.sort.ColumnarShuffleManager;
+import org.apache.spark.shuffle.writer.VeloxUniffleColumnarShuffleWriter;
+import org.apache.uniffle.common.exception.RssException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+
+public class GlutenRssShuffleManager extends RssShuffleManager {

Review Comment:
   Follow this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-08 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2044161671

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-08 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2044075707

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-08 Thread via GitHub


PHILO-HE commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1556751289


##
cpp/core/jni/JniWrapper.cc:
##
@@ -915,6 +915,23 @@ JNIEXPORT jlong JNICALL 
Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe
 std::move(partitionWriterOptions),
 memoryManager->getArrowMemoryPool(),
 std::move(celebornClient));
+  } else if (partitionWriterType == "uniffle") {
+jclass unifflePartitionPusherClass =
+createGlobalClassReferenceOrError(env, 
"Lorg/apache/spark/shuffle/writer/PartitionPusher;");
+jmethodID unifflePushPartitionDataMethod =
+getMethodIdOrError(env, unifflePartitionPusherClass, 
"pushPartitionData", "(I[BI)I");
+JavaVM* vm;
+if (env->GetJavaVM() != JNI_OK) {
+  throw gluten::GlutenException("Unable to get JavaVM instance");
+}
+// rename CelebornClient RssClient
+std::shared_ptr uniffleClient =
+std::make_shared(vm, partitionPusher, 
unifflePushPartitionDataMethod);
+partitionWriter = std::make_unique(

Review Comment:
   @summaryzb, ok to me. Please create an issue to track this. Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-08 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2044021508

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-08 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1556740563


##
cpp/core/jni/JniWrapper.cc:
##
@@ -915,6 +915,23 @@ JNIEXPORT jlong JNICALL 
Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe
 std::move(partitionWriterOptions),
 memoryManager->getArrowMemoryPool(),
 std::move(celebornClient));
+  } else if (partitionWriterType == "uniffle") {
+jclass unifflePartitionPusherClass =
+createGlobalClassReferenceOrError(env, 
"Lorg/apache/spark/shuffle/writer/PartitionPusher;");
+jmethodID unifflePushPartitionDataMethod =
+getMethodIdOrError(env, unifflePartitionPusherClass, 
"pushPartitionData", "(I[BI)I");
+JavaVM* vm;
+if (env->GetJavaVM() != JNI_OK) {
+  throw gluten::GlutenException("Unable to get JavaVM instance");
+}
+// rename CelebornClient RssClient
+std::shared_ptr uniffleClient =
+std::make_shared(vm, partitionPusher, 
unifflePushPartitionDataMethod);
+partitionWriter = std::make_unique(

Review Comment:
   Using a common name relate to all the logic of rss in jni layer, but not 
limit to this `PartitionWriter`, maybe it's better to resolve this in a 
separate pr



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-07 Thread via GitHub


PHILO-HE commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1554839641


##
.github/workflows/velox_docker.yml:
##
@@ -275,6 +276,66 @@ jobs:
   #   -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \
   #   -d=OFFHEAP_SIZE:1g,spark.memory.offHeap.size=1g || true
 
+  run-tpc-test-centos8-uniffle:
+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 git
+  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 clean install -P${{ matrix.spark }} -Pbackends-velox 
-Prss-uniffle -DskipTests
+  - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle 
0.8.0
+run: |
+  export MAVEN_HOME=/usr/lib/maven && \
+  export PATH=${PATH}:${MAVEN_HOME}/bin && \
+  export export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \
+  cd /opt && \
+  git clone -b branch-0.8 
https://github.com/apache/incubator-uniffle.git && \
+  cd incubator-uniffle && \
+  sed -i '250d' 
./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \
+  sed -i '228d' 
./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \
+  sed -i '226d' 
./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \
+  mvn clean install -Phadoop2.8 -DskipTests
+  cd /opt && \
+  wget -nv 
https://archive.apache.org/dist/incubator/uniffle/0.8.0/apache-uniffle-0.8.0-incubating-bin.tar.gz
 && \
+  tar xzf apache-uniffle-0.8.0-incubating-bin.tar.gz -C /opt/ && mv 
/opt/rss-0.8.0-hadoop2.8 /opt/uniffle && \
+  wget -nv 
https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz 
&& \
+  tar xzf hadoop-2.8.5.tar.gz -C /opt/
+  rm -f /opt/uniffle/jars/server/shuffle-server-0.8.0-SNAPSHOT.jar
+  cp 
/opt/incubator-uniffle/server/target/shuffle-server-0.8.1-SNAPSHOT.jar 
/opt/uniffle/jars/server/
+  rm -rf /opt/incubator-uniffle
+  cd /opt/uniffle && mkdir shuffle_data && \
+  echo -e "XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-2.8.5" > 
./bin/rss-env.sh && \

Review Comment:
   I found "echo -e" requires to be executed by `bash`, and it's NOT `sh` 
compliant. So let's explicitly use `bash` like this:
   
https://github.com/apache/incubator-gluten/blob/main/.github/workflows/velox_docker.yml#L363



##
cpp/core/jni/JniWrapper.cc:
##
@@ -915,6 +915,23 @@ JNIEXPORT jlong JNICALL 
Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe
 std::move(partitionWriterOptions),
 memoryManager->getArrowMemoryPool(),
 std::move(celebornClient));
+  } else if (partitionWriterType == "uniffle") {
+jclass unifflePartitionPusherClass =
+createGlobalClassReferenceOrError(env, 
"Lorg/apache/spark/shuffle/writer/PartitionPusher;");
+jmethodID unifflePushPartitionDataMethod =
+getMethodIdOrError(env, unifflePartitionPusherClass, 
"pushPartitionData", "(I[BI)I");
+JavaVM* vm;
+if (env->GetJavaVM() != JNI_OK) {
+  throw gluten::GlutenException("Unable to get JavaVM instance");
+}
+// rename CelebornClient RssClient
+std::shared_ptr uniffleClient =
+std::make_shared(vm, partitionPusher, 
unifflePushPartitionDataMethod);
+partitionWriter = std::make_unique(

Review Comment:
   Like the above comment, better to use a common name if they indeed can be 
shared by both celeborn & uniffle.



##
gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/gluten/uniffle/GlutenRssShuffleManager.java:
##
@@ -0,0 +1,140 @@
+/*
+ * 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 

Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-05 Thread via GitHub


summaryzb commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2040846636

   @jackylee-ch PTAL


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-03 Thread via GitHub


summaryzb commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2034387196

   Uniffle run tpcds with only one worker may encounter some stability issues 
occasionally,so this integration test only run with tpch


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-03 Thread via GitHub


summaryzb commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2034381626

   @PHILO-HE PTAL


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-03 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2034018894

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-03 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2034018443

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-03 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2033667426

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-02 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2031916793

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-02 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2031767689

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-02 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2031621223

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-02 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2031429130

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-02 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2031275066

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-01 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2031130447

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-01 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2030953998

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-01 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2029587355

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-04-01 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2029478000

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-31 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2029054476

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-29 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2026928568

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-29 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2026830719

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-29 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2026769020

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-27 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2022203787

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-27 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2022045396

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-26 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2021937111

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-26 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2021856789

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-25 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2017371170

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-25 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2017323489

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-25 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2017307134

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-22 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2014490557

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-21 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2014196489

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-21 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2011372622

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-20 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2011029672

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-20 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2009497149

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-20 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1532020728


##
.github/workflows/velox_be.yml:
##
@@ -441,7 +442,37 @@ jobs:
 run: |
   $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh '
   cd /opt/gluten && \
-  mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Piceberg 
-Pdelta -DskipTests'
+  mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Prss-uniffle 
-Piceberg -Pdelta -DskipTests'
+  - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.2 with uniffle
+run: |
+  $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh '
+  echo "java path:`command -v java`" && \
+  echo "cur dir `pwd`" && \
+  wget -nv 
https://archive.apache.org/dist/incubator/uniffle/0.8.0/apache-uniffle-0.8.0-incubating-bin.tar.gz
 && \
+  tar xzf apache-uniffle-0.8.0-incubating-bin.tar.gz -C /opt/ && mv 
/opt/rss-0.8.0-hadoop2.8 /opt/uniffle && \
+  wget -nv 
https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz 
&& \
+  tar xzf hadoop-2.8.5.tar.gz -C /opt/ && \
+  cd /opt/uniffle && mkdir shuffle_data && \
+  echo -e 
"XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-2.8.5\nJAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64"
 > ./bin/rss-env.sh && \
+  echo -e "rss.coordinator.server.periodic.output.interval.times 
1\nrss.coordinator.shuffle.nodes.max 1" > ./conf/coordinator.conf && \
+  echo -e "rss.server.heartbeat.delay 3000\nrss.rpc.server.port 
19997\nrss.jetty.http.port 19996\nrss.server.netty.port 
19995\nrss.storage.basePath /opt/uniffle/shuffle_data\nrss.storage.type 
MEMORY_LOCALFILE\nrss.coordinator.quorum 
localhost:1\nrss.server.flush.thread.alive 
5\nrss.server.single.buffer.flush.threshold 64m" > ./conf/server.conf && \
+  bash ./bin/start-coordinator.sh && bash 
./bin/start-shuffle-server.sh && \
+  sleep 50 && \
+  echo "=show uniffle startup" && \
+  cat ./logs/coordinator.log && \
+  cat ./logs/shuffle_server.log && \
+  cd /opt/gluten/tools/gluten-it && mvn clean install 
-Pspark-3.2,rss-uniffle \
+  && tail -n 200 /opt/uniffle/logs/coordinator.log \
+  && tail -n 200 /opt/uniffle/logs/shuffle_server.log \
+  && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
+--local --preset=velox-with-uniffle --benchmark-type=h 
--error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \
+#  && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
+#--local --preset=velox-with-uniffle --benchmark-type=ds 
--error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \

Review Comment:
   thanks for reminding



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-20 Thread via GitHub


PHILO-HE commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1531988230


##
.github/workflows/velox_be.yml:
##
@@ -441,7 +442,37 @@ jobs:
 run: |
   $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh '
   cd /opt/gluten && \
-  mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Piceberg 
-Pdelta -DskipTests'
+  mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Prss-uniffle 
-Piceberg -Pdelta -DskipTests'
+  - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.2 with uniffle
+run: |
+  $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh '
+  echo "java path:`command -v java`" && \
+  echo "cur dir `pwd`" && \
+  wget -nv 
https://archive.apache.org/dist/incubator/uniffle/0.8.0/apache-uniffle-0.8.0-incubating-bin.tar.gz
 && \
+  tar xzf apache-uniffle-0.8.0-incubating-bin.tar.gz -C /opt/ && mv 
/opt/rss-0.8.0-hadoop2.8 /opt/uniffle && \
+  wget -nv 
https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz 
&& \
+  tar xzf hadoop-2.8.5.tar.gz -C /opt/ && \
+  cd /opt/uniffle && mkdir shuffle_data && \
+  echo -e 
"XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-2.8.5\nJAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64"
 > ./bin/rss-env.sh && \
+  echo -e "rss.coordinator.server.periodic.output.interval.times 
1\nrss.coordinator.shuffle.nodes.max 1" > ./conf/coordinator.conf && \
+  echo -e "rss.server.heartbeat.delay 3000\nrss.rpc.server.port 
19997\nrss.jetty.http.port 19996\nrss.server.netty.port 
19995\nrss.storage.basePath /opt/uniffle/shuffle_data\nrss.storage.type 
MEMORY_LOCALFILE\nrss.coordinator.quorum 
localhost:1\nrss.server.flush.thread.alive 
5\nrss.server.single.buffer.flush.threshold 64m" > ./conf/server.conf && \
+  bash ./bin/start-coordinator.sh && bash 
./bin/start-shuffle-server.sh && \
+  sleep 50 && \
+  echo "=show uniffle startup" && \
+  cat ./logs/coordinator.log && \
+  cat ./logs/shuffle_server.log && \
+  cd /opt/gluten/tools/gluten-it && mvn clean install 
-Pspark-3.2,rss-uniffle \
+  && tail -n 200 /opt/uniffle/logs/coordinator.log \
+  && tail -n 200 /opt/uniffle/logs/shuffle_server.log \
+  && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
+--local --preset=velox-with-uniffle --benchmark-type=h 
--error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \
+#  && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
+#--local --preset=velox-with-uniffle --benchmark-type=ds 
--error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \

Review Comment:
   @summaryzb, looks the yml file is ill-formed which causes CI not triggered. 
You can remove line 469 and 470. cc @weiting-chen 
   P.S., you can verify a yml file in https://www.yamllint.com/.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-20 Thread via GitHub


PHILO-HE commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1531988230


##
.github/workflows/velox_be.yml:
##
@@ -441,7 +442,37 @@ jobs:
 run: |
   $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh '
   cd /opt/gluten && \
-  mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Piceberg 
-Pdelta -DskipTests'
+  mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Prss-uniffle 
-Piceberg -Pdelta -DskipTests'
+  - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.2 with uniffle
+run: |
+  $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh '
+  echo "java path:`command -v java`" && \
+  echo "cur dir `pwd`" && \
+  wget -nv 
https://archive.apache.org/dist/incubator/uniffle/0.8.0/apache-uniffle-0.8.0-incubating-bin.tar.gz
 && \
+  tar xzf apache-uniffle-0.8.0-incubating-bin.tar.gz -C /opt/ && mv 
/opt/rss-0.8.0-hadoop2.8 /opt/uniffle && \
+  wget -nv 
https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz 
&& \
+  tar xzf hadoop-2.8.5.tar.gz -C /opt/ && \
+  cd /opt/uniffle && mkdir shuffle_data && \
+  echo -e 
"XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-2.8.5\nJAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64"
 > ./bin/rss-env.sh && \
+  echo -e "rss.coordinator.server.periodic.output.interval.times 
1\nrss.coordinator.shuffle.nodes.max 1" > ./conf/coordinator.conf && \
+  echo -e "rss.server.heartbeat.delay 3000\nrss.rpc.server.port 
19997\nrss.jetty.http.port 19996\nrss.server.netty.port 
19995\nrss.storage.basePath /opt/uniffle/shuffle_data\nrss.storage.type 
MEMORY_LOCALFILE\nrss.coordinator.quorum 
localhost:1\nrss.server.flush.thread.alive 
5\nrss.server.single.buffer.flush.threshold 64m" > ./conf/server.conf && \
+  bash ./bin/start-coordinator.sh && bash 
./bin/start-shuffle-server.sh && \
+  sleep 50 && \
+  echo "=show uniffle startup" && \
+  cat ./logs/coordinator.log && \
+  cat ./logs/shuffle_server.log && \
+  cd /opt/gluten/tools/gluten-it && mvn clean install 
-Pspark-3.2,rss-uniffle \
+  && tail -n 200 /opt/uniffle/logs/coordinator.log \
+  && tail -n 200 /opt/uniffle/logs/shuffle_server.log \
+  && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
+--local --preset=velox-with-uniffle --benchmark-type=h 
--error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \
+#  && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
+#--local --preset=velox-with-uniffle --benchmark-type=ds 
--error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \

Review Comment:
   @summaryzb, looks the yml file is ill-formed, which causes CI not triggered. 
You can remove line 469 and 470. cc @weiting-chen 
   P.S., you can verify a yml file in https://www.yamllint.com/.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-20 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2009415207

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-18 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2005655391

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-18 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2005626053

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-18 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2005625596

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-18 Thread via GitHub


summaryzb closed pull request #3767: [VL] Add uniffle integration
URL: https://github.com/apache/incubator-gluten/pull/3767


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-18 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2003366868

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-18 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2003038194

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-17 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-2002785304

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-15 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1999362810

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-15 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1999187799

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-15 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1999125019

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-15 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1999060161

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-15 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1999010955

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-14 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1997288825

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1994491935

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1994395911

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1994392244

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1994224790

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1994222094

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1994186495

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993951021

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993730910

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993665691

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993645415

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993633440

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-13 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993630983

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-12 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993626916

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-12 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993614839

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-12 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993383531

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-12 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993290413

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-12 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993257271

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-12 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1993139416

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-12 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1990970382

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-11 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1990169436

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-11 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1989951371

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-11 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1988347941

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-11 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1988274981

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-11 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1987875825

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-10 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1987677417

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-10 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1987660382

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-10 Thread via GitHub


summaryzb commented on code in PR #3767:
URL: https://github.com/apache/incubator-gluten/pull/3767#discussion_r1519167787


##
pom.xml:
##
@@ -50,6 +50,7 @@
 
spark-sql-columnar-shims-spark32
 0.3.0-incubating
 14.0.1
+0.8.0

Review Comment:
   Only `spark.major.version` matters, it's defined already in gluten



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-10 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1987648463

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-10 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1987648467

   Run Gluten Clickhouse CI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-10 Thread via GitHub


summaryzb closed pull request #3767: [VL] Add uniffle integration
URL: https://github.com/apache/incubator-gluten/pull/3767


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



Re: [PR] [VL] Add uniffle integration [incubator-gluten]

2024-03-10 Thread via GitHub


github-actions[bot] commented on PR #3767:
URL: 
https://github.com/apache/incubator-gluten/pull/3767#issuecomment-1987482940

   This PR is stale because it has been open 45 days with no activity. Remove 
stale label or comment or this will be closed in 10 days.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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