[GitHub] [spark] cloud-fan commented on a change in pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


cloud-fan commented on a change in pull request #33803:
URL: https://github.com/apache/spark/pull/33803#discussion_r704071320



##
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/In.java
##
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.expressions.filter;
+
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.Literal;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+
+/**
+ * A filter that evaluates to `true` iff the column evaluates to one of the 
values in the array.
+ *
+ * @since 3.3.0
+ */
+@Evolving
+public final class In extends Filter {
+  private final NamedReference column;
+  private final Literal[] values;
+
+  public In(NamedReference column, Literal[] values) {

Review comment:
   Good point. It should not. But there is no `private[sql]` in Java ...




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] LuciferYang opened a new pull request #33933: [SPARK-36690][SS] Clean up deprecated api usage after upgrade commons-pool2 to 2.11.1

2021-09-07 Thread GitBox


LuciferYang opened a new pull request #33933:
URL: https://github.com/apache/spark/pull/33933


   ### What changes were proposed in this pull request?
   SPARK-36583 upgrade `Apache commons-pool2` from 2.6.2 to 2.11.1 and there 
are some deprecated API usage related to it that need to be cleaned up.
   
   The list of changes is as follows:
   
   - `BaseObjectPoolConfig.setMinEvictableIdleTimeMillis` -> 
`BaseObjectPoolConfig.setMinEvictableIdleTime`
   - `BaseObjectPoolConfig.setSoftMinEvictableIdleTimeMillis` -> 
`BaseObjectPoolConfig.setSoftMinEvictableIdleTime`
   - `BaseObjectPoolConfig.setTimeBetweenEvictionRunsMillis` -> 
`BaseObjectPoolConfig.setTimeBetweenEvictionRuns`
   
   
   ### Why are the changes needed?
   Clean up deprecated API  usage.
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Pass GA or Jenkins Tests.
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] LuciferYang commented on pull request #33926: [SPARK-36684][SQL][TESTS] Add Jackson test dependencies to `sql/core` module at `hadoop-2.7` profile

2021-09-07 Thread GitBox


LuciferYang commented on pull request #33926:
URL: https://github.com/apache/spark/pull/33926#issuecomment-914941096


   thanks @dongjoon-hyun and @sunchao 


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on a change in pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


HyukjinKwon commented on a change in pull request #33803:
URL: https://github.com/apache/spark/pull/33803#discussion_r704070307



##
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/In.java
##
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.expressions.filter;
+
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.Literal;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+
+/**
+ * A filter that evaluates to `true` iff the column evaluates to one of the 
values in the array.
+ *
+ * @since 3.3.0
+ */
+@Evolving
+public final class In extends Filter {
+  private final NamedReference column;
+  private final Literal[] values;
+
+  public In(NamedReference column, Literal[] values) {

Review comment:
   should the constructor be public?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on a change in pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


HyukjinKwon commented on a change in pull request #33803:
URL: https://github.com/apache/spark/pull/33803#discussion_r704069808



##
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/AlwaysTrue.java
##
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.expressions.filter;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+
+/**
+ * A filter that always evaluates to `true`.

Review comment:
   Let's use `{@code}` instead of backquotes.




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


SparkQA commented on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914939272


   Kubernetes integration test status failure
   URL: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47568/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on a change in pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


HyukjinKwon commented on a change in pull request #33803:
URL: https://github.com/apache/spark/pull/33803#discussion_r704069445



##
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/EqualNullSafe.java
##
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.expressions.filter;
+
+import java.util.Objects;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.Literal;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+
+/**
+ * Performs equality comparison, similar to [[EqualTo]]. However, this differs 
from [[EqualTo]]

Review comment:
   Can we use Javadoc style of class reference instead of `[[..]]`?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


SparkQA commented on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914939020


   **[Test build #143068 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143068/testReport)**
 for PR 33930 at commit 
[`5600207`](https://github.com/apache/spark/commit/5600207665f0cf3b4deeaf7e4665a8eb10beb724).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


SparkQA commented on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914939044


   **[Test build #143066 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143066/testReport)**
 for PR 33911 at commit 
[`f872943`](https://github.com/apache/spark/commit/f872943d225afbdc0e458d0ace4e8227a723a166).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds no public classes.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on a change in pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


HyukjinKwon commented on a change in pull request #33803:
URL: https://github.com/apache/spark/pull/33803#discussion_r704066217



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2FiltersSuite.scala
##
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.connector.expressions.{FieldReference, 
LiteralValue}
+import org.apache.spark.sql.connector.expressions.filter._
+import org.apache.spark.sql.execution.datasources.v2.FiltersV2Suite.ref
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.unsafe.types.UTF8String
+
+class FiltersV2Suite extends SparkFunSuite {
+
+  test("nested columns") {
+val filter1 = new EqualTo(ref("a", "B"), LiteralValue(1, IntegerType))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a.B"))
+assert(filter1.describe.equals("a.B = 1"))
+
+val filter2 = new EqualTo(ref("a", "b.c"), LiteralValue(1, IntegerType))
+assert(filter2.references.map(_.describe()).toSeq == Seq("a.`b.c`"))
+assert(filter2.describe.equals("a.`b.c` = 1"))
+
+val filter3 = new EqualTo(ref("`a`.b", "c"), LiteralValue(1, IntegerType))
+assert(filter3.references.map(_.describe()).toSeq == Seq("```a``.b`.c"))
+assert(filter3.describe.equals("```a``.b`.c = 1"))
+  }
+
+  test("EqualTo") {
+val filter1 = new EqualTo(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new EqualTo(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a = 1"))
+  }
+
+  test("EqualNullSafe") {
+val filter1 = new EqualNullSafe(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new EqualNullSafe(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a <=> 1"))
+  }
+
+  test("GreaterThan") {
+val filter1 = new GreaterThan(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new GreaterThan(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a > 1"))
+  }
+
+  test("GreaterThanOrEqual") {
+val filter1 = new GreaterThanOrEqual(ref("a"), LiteralValue(1, 
IntegerType))
+val filter2 = new GreaterThanOrEqual(ref("a"), LiteralValue(1, 
IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a >= 1"))
+  }
+
+  test("LessThan") {
+val filter1 = new LessThan(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new LessThan(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a < 1"))
+  }
+
+  test("LessThanOrEqual") {
+val filter1 = new LessThanOrEqual(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new LessThanOrEqual(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a <= 1"))
+  }
+
+  test("In") {
+val filter1 = new In(ref("a"),
+  Array(LiteralValue(1, IntegerType), LiteralValue(2, IntegerType),
+LiteralValue(3, IntegerType), LiteralValue(4, IntegerType)))
+val filter2 = new In(ref("a"),
+  Array(LiteralValue(1, IntegerType), LiteralValue(2, IntegerType),
+LiteralValue(3, IntegerType), LiteralValue(4, IntegerType)))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a IN (1, 2, 3, 4)"))
+  }
+
+  test("IsNull") {
+val filter1 = new IsNull(ref("a"))
+val filter2 = new IsNull(ref("a"))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+

[GitHub] [spark] Ngone51 commented on pull request #33872: [SPARK-36575][CORE] Should ignore task finished event if its task set is gone in TaskSchedulerImpl.handleSuccessfulTask

2021-09-07 Thread GitBox


Ngone51 commented on pull request #33872:
URL: https://github.com/apache/spark/pull/33872#issuecomment-914934081


   @mridulm As we discussed at 
https://github.com/apache/spark/pull/33872#discussion_r703279670, I think me 
and @sleep1661 have reached the agreement that the hang issue exists in 2.3 but 
not master.  So, now, I think our target is to fix the inconsistent status of 
the task(I'm actually not sure if the issue really exists because I didn't pay 
much attention on this before. I asked @sleep1661 to update the PR description 
first. So let's see.).  To clarify, I think Spark actually can heal the 
inconsistent issue by rerun the stage later. But if we could fix it early in 
TaskSetManager, then, we can fix the issue eariler with less cost.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on pull request #33929: [SPARK-36618] Support dropping rows of a single-indexed DataFrame

2021-09-07 Thread GitBox


HyukjinKwon commented on pull request #33929:
URL: https://github.com/apache/spark/pull/33929#issuecomment-914933856


   cc @ueshin @itholic FYI


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33872: [SPARK-36575][CORE] Should ignore task finished event if its task set is gone in TaskSchedulerImpl.handleSuccessfulTask

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33872:
URL: https://github.com/apache/spark/pull/33872#issuecomment-910840266


   
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/47435/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] Ngone51 commented on a change in pull request #33872: [SPARK-36575][CORE] Should ignore task finished event if its task set is gone in TaskSchedulerImpl.handleSuccessfulTask

2021-09-07 Thread GitBox


Ngone51 commented on a change in pull request #33872:
URL: https://github.com/apache/spark/pull/33872#discussion_r704059869



##
File path: 
core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
##
@@ -1995,6 +2000,61 @@ class TaskSchedulerImplSuite extends SparkFunSuite with 
LocalSparkContext with B
 assert(!normalTSM.runningTasksSet.contains(taskId))
   }
 
+  test("SPARK-36575: Executor lost cause task hang") {
+val taskScheduler = setupScheduler()
+
+val resultGetter = new TaskResultGetter(sc.env, taskScheduler) {
+  override protected val getTaskResultExecutor: ExecutorService =
+ThreadUtils.newDaemonFixedThreadPool(1, "task-result-getter")
+  def taskResultExecutor() : ExecutorService = getTaskResultExecutor
+}
+taskScheduler.taskResultGetter = resultGetter
+
+val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 1),
+  new WorkerOffer("executor1", "host1", 1))
+val task1 = new ShuffleMapTask(1, 0, null, new Partition {
+  override def index: Int = 0
+}, Seq(TaskLocation("host0", "executor0")), new Properties, null)
+
+val task2 = new ShuffleMapTask(1, 0, null, new Partition {
+  override def index: Int = 0
+}, Seq(TaskLocation("host1", "executor1")), new Properties, null)
+
+val taskSet = new TaskSet(Array(task1, task2), 0, 0, 0, null, 0)
+
+taskScheduler.submitTasks(taskSet)
+val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+assert(2 === taskDescriptions.length)
+
+val ser = sc.env.serializer.newInstance()
+val directResult = new DirectTaskResult[Int](ser.serialize(1), Seq(), 
Array.empty)
+val resultBytes = ser.serialize(directResult)
+
+// make getTaskResultExecutor busy
+import scala.language.reflectiveCalls
+resultGetter.taskResultExecutor().submit( new Runnable {
+  override def run(): Unit = Thread.sleep(100)
+})
+
+// task1 finished
+taskScheduler.statusUpdate(
+  tid = taskDescriptions(0).taskId,
+  state = TaskState.FINISHED,
+  serializedData = resultBytes
+)
+
+// mark executor heartbeat timed out
+taskScheduler.executorLost(taskDescriptions(0).executorId, 
ExecutorProcessLost("Executor " +
+  "heartbeat timed out"))
+
+// Wait a while until all events are processed
+Thread.sleep(100)

Review comment:
   Yea, but could you rephrase the PR description first?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


SparkQA commented on pull request #33803:
URL: https://github.com/apache/spark/pull/33803#issuecomment-914928927


   **[Test build #143067 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143067/testReport)**
 for PR 33803 at commit 
[`3c200a5`](https://github.com/apache/spark/commit/3c200a5ed00dd6da3365964635fbba2a793291cc).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33931:
URL: https://github.com/apache/spark/pull/33931#issuecomment-914928467






-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


SparkQA commented on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914928745


   **[Test build #143066 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143066/testReport)**
 for PR 33911 at commit 
[`f872943`](https://github.com/apache/spark/commit/f872943d225afbdc0e458d0ace4e8227a723a166).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33932: [SPARK-33781][SHUFFLE] Improve caching of MergeStatus on the executor side to save memory

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33932:
URL: https://github.com/apache/spark/pull/33932#issuecomment-914928547


   Can one of the admins verify this patch?


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33931:
URL: https://github.com/apache/spark/pull/33931#issuecomment-914928468






-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on pull request #33928: [SPARK-36686][SQL] Fix SimplifyConditionalsInPredicate to be null-safe

2021-09-07 Thread GitBox


HyukjinKwon commented on pull request #33928:
URL: https://github.com/apache/spark/pull/33928#issuecomment-914927592


   Also, Apache Spark uses forked repository's resource. Please enable Github 
Actions in your forked repository (see also 
https://github.com/apache/spark/pull/33928/checks?check_run_id=3537007064)


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on pull request #33928: [SPARK-36686][SQL] Fix SimplifyConditionalsInPredicate to be null-safe

2021-09-07 Thread GitBox


HyukjinKwon commented on pull request #33928:
URL: https://github.com/apache/spark/pull/33928#issuecomment-914927440


   Can you add a test and show the reproducer with incorrect results in the PR 
description?


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


SparkQA commented on pull request #33931:
URL: https://github.com/apache/spark/pull/33931#issuecomment-914927332


   Kubernetes integration test status failure
   URL: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47567/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] HyukjinKwon closed pull request #33925: [SPARK-36688][R] Add cot as an R function

2021-09-07 Thread GitBox


HyukjinKwon closed pull request #33925:
URL: https://github.com/apache/spark/pull/33925


   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] HyukjinKwon commented on pull request #33925: [SPARK-36688][R] Add cot as an R function

2021-09-07 Thread GitBox


HyukjinKwon commented on pull request #33925:
URL: https://github.com/apache/spark/pull/33925#issuecomment-914926027


   Merged to master.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA removed a comment on pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


SparkQA removed a comment on pull request #33931:
URL: https://github.com/apache/spark/pull/33931#issuecomment-914903403


   **[Test build #143064 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143064/testReport)**
 for PR 33931 at commit 
[`de6d1fd`](https://github.com/apache/spark/commit/de6d1fdc7e7769c0eaa136e7df4fda24e0eda5e1).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] SparkQA commented on pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


SparkQA commented on pull request #33931:
URL: https://github.com/apache/spark/pull/33931#issuecomment-914924134


   Kubernetes integration test starting
   URL: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47567/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


SparkQA commented on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914923853


   Kubernetes integration test starting
   URL: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47568/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] gengliangwang commented on a change in pull request #33919: [SPARK-36674][SQL] Support ILIKE - case insensitive LIKE

2021-09-07 Thread GitBox


gengliangwang commented on a change in pull request #33919:
URL: https://github.com/apache/spark/pull/33919#discussion_r704052590



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -121,70 +187,73 @@ abstract class StringRegexExpression extends 
BinaryExpression
   since = "1.0.0",
   group = "predicate_funcs")
 // scalastyle:on line.contains.tab
-case class Like(left: Expression, right: Expression, escapeChar: Char)
-  extends StringRegexExpression {
+case class Like(left: Expression, right: Expression, escapeChar: Char) extends 
LikeBase {
 
   def this(left: Expression, right: Expression) = this(left, right, '\\')
 
-  override def escape(v: String): String = StringUtils.escapeLikeRegex(v, 
escapeChar)
+  override protected def likeName: String = "LIKE"
 
-  override def matches(regex: Pattern, str: String): Boolean = 
regex.matcher(str).matches()
+  override protected def withNewChildrenInternal(newLeft: Expression, 
newRight: Expression): Like =
+copy(left = newLeft, right = newRight)
+}
 
-  final override val nodePatterns: Seq[TreePattern] = Seq(LIKE_FAMLIY)
+// scalastyle:off line.contains.tab
+/**
+ * Simple RegEx case-insensitive pattern matching function
+ */
+@ExpressionDescription(
+  usage = "str _FUNC_ pattern[ ESCAPE escape] - Returns true if str matches 
`pattern` with " +
+"`escape` case-insensitively, null if any arguments are null, false 
otherwise.",
+  arguments = """
+Arguments:
+  * str - a string expression
+  * pattern - a string expression. The pattern is a string which is 
matched literally and
+  case-insensitively, with exception to the following special symbols:
 
-  override def toString: String = escapeChar match {
-case '\\' => s"$left LIKE $right"
-case c => s"$left LIKE $right ESCAPE '$c'"
-  }
+  _ matches any one character in the input (similar to . in posix 
regular expressions)
 
-  override def sql: String = s"${left.sql} 
${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}"
+  % matches zero or more characters in the input (similar to .* in 
posix regular
+  expressions)
 
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
-val patternClass = classOf[Pattern].getName
-val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
+  Since Spark 2.0, string literals are unescaped in our SQL parser. 
For example, in order
+  to match "\abc", the pattern should be "\\abc".
 
-if (right.foldable) {
-  val rVal = right.eval()
-  if (rVal != null) {
-val regexStr =
-  
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-val pattern = ctx.addMutableState(patternClass, "patternLike",
-  v => s"""$v = $patternClass.compile("$regexStr");""")
+  When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, 
it falls back
+  to Spark 1.6 behavior regarding string literal parsing. For example, 
if the config is
+  enabled, the pattern to match "\abc" should be "\abc".
+  * escape - an character added since Spark 3.0. The default escape 
character is the '\'.
+  If an escape character precedes a special symbol or another escape 
character, the
+  following character is matched literally. It is invalid to escape 
any other character.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_('Spark', '_Park');
+  true
+  > SET spark.sql.parser.escapedStringLiterals=true;

Review comment:
   It seems odd to use these examples as test cases for the behavior. 
Anyway, this is a minor comment.




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] MaxGekk commented on a change in pull request #33919: [SPARK-36674][SQL] Support ILIKE - case insensitive LIKE

2021-09-07 Thread GitBox


MaxGekk commented on a change in pull request #33919:
URL: https://github.com/apache/spark/pull/33919#discussion_r704049382



##
File path: 
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##
@@ -798,7 +798,7 @@ predicate
 | NOT? kind=IN '(' query ')'
 | NOT? kind=RLIKE pattern=valueExpression
 | NOT? kind=LIKE quantifier=(ANY | SOME | ALL) ('('')' | '(' expression 
(',' expression)* ')')

Review comment:
   See https://docs.snowflake.com/en/sql-reference/functions/ilike_any.html




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] MaxGekk commented on a change in pull request #33919: [SPARK-36674][SQL] Support ILIKE - case insensitive LIKE

2021-09-07 Thread GitBox


MaxGekk commented on a change in pull request #33919:
URL: https://github.com/apache/spark/pull/33919#discussion_r704049052



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -121,70 +187,73 @@ abstract class StringRegexExpression extends 
BinaryExpression
   since = "1.0.0",
   group = "predicate_funcs")
 // scalastyle:on line.contains.tab
-case class Like(left: Expression, right: Expression, escapeChar: Char)
-  extends StringRegexExpression {
+case class Like(left: Expression, right: Expression, escapeChar: Char) extends 
LikeBase {
 
   def this(left: Expression, right: Expression) = this(left, right, '\\')
 
-  override def escape(v: String): String = StringUtils.escapeLikeRegex(v, 
escapeChar)
+  override protected def likeName: String = "LIKE"
 
-  override def matches(regex: Pattern, str: String): Boolean = 
regex.matcher(str).matches()
+  override protected def withNewChildrenInternal(newLeft: Expression, 
newRight: Expression): Like =
+copy(left = newLeft, right = newRight)
+}
 
-  final override val nodePatterns: Seq[TreePattern] = Seq(LIKE_FAMLIY)
+// scalastyle:off line.contains.tab
+/**
+ * Simple RegEx case-insensitive pattern matching function
+ */
+@ExpressionDescription(
+  usage = "str _FUNC_ pattern[ ESCAPE escape] - Returns true if str matches 
`pattern` with " +
+"`escape` case-insensitively, null if any arguments are null, false 
otherwise.",
+  arguments = """
+Arguments:
+  * str - a string expression
+  * pattern - a string expression. The pattern is a string which is 
matched literally and
+  case-insensitively, with exception to the following special symbols:
 
-  override def toString: String = escapeChar match {
-case '\\' => s"$left LIKE $right"
-case c => s"$left LIKE $right ESCAPE '$c'"
-  }
+  _ matches any one character in the input (similar to . in posix 
regular expressions)
 
-  override def sql: String = s"${left.sql} 
${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}"
+  % matches zero or more characters in the input (similar to .* in 
posix regular
+  expressions)
 
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
-val patternClass = classOf[Pattern].getName
-val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
+  Since Spark 2.0, string literals are unescaped in our SQL parser. 
For example, in order
+  to match "\abc", the pattern should be "\\abc".
 
-if (right.foldable) {
-  val rVal = right.eval()
-  if (rVal != null) {
-val regexStr =
-  
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-val pattern = ctx.addMutableState(patternClass, "patternLike",
-  v => s"""$v = $patternClass.compile("$regexStr");""")
+  When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, 
it falls back
+  to Spark 1.6 behavior regarding string literal parsing. For example, 
if the config is
+  enabled, the pattern to match "\abc" should be "\abc".
+  * escape - an character added since Spark 3.0. The default escape 
character is the '\'.
+  If an escape character precedes a special symbol or another escape 
character, the
+  following character is matched literally. It is invalid to escape 
any other character.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_('Spark', '_Park');
+  true
+  > SET spark.sql.parser.escapedStringLiterals=true;

Review comment:
   The examples are executed in a test too. Or do you want to test 
something specifically to case-sensitivity?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] cloud-fan commented on a change in pull request #33919: [SPARK-36674][SQL] Support ILIKE - case insensitive LIKE

2021-09-07 Thread GitBox


cloud-fan commented on a change in pull request #33919:
URL: https://github.com/apache/spark/pull/33919#discussion_r704048628



##
File path: 
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
##
@@ -798,7 +798,7 @@ predicate
 | NOT? kind=IN '(' query ')'
 | NOT? kind=RLIKE pattern=valueExpression
 | NOT? kind=LIKE quantifier=(ANY | SOME | ALL) ('('')' | '(' expression 
(',' expression)* ')')

Review comment:
   do other RDBMS support ILIKE ANY?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704046963



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.

Review comment:
   Please refer to the 
https://github.com/apache/spark/pull/33858#discussion_r704045118.




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] rmcyang commented on pull request #33932: [Spark-33781][SHUFFLE] Improve caching of MergeStatus on the executor side to save memory

2021-09-07 Thread GitBox


rmcyang commented on pull request #33932:
URL: https://github.com/apache/spark/pull/33932#issuecomment-914914201


   cc @Ngone51 @dongjoon-hyun @Victsm @otterc @mridulm @venkata91 @zhouyejoe 
   Please take a look.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] SparkQA commented on pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


SparkQA commented on pull request #33931:
URL: https://github.com/apache/spark/pull/33931#issuecomment-914913850


   **[Test build #143064 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143064/testReport)**
 for PR 33931 at commit 
[`de6d1fd`](https://github.com/apache/spark/commit/de6d1fdc7e7769c0eaa136e7df4fda24e0eda5e1).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds no public classes.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] cloud-fan commented on a change in pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


cloud-fan commented on a change in pull request #33803:
URL: https://github.com/apache/spark/pull/33803#discussion_r704045703



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2FiltersSuite.scala
##
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.connector.expressions.{FieldReference, 
LiteralValue}
+import org.apache.spark.sql.connector.expressions.filter._
+import org.apache.spark.sql.execution.datasources.v2.FiltersV2Suite.ref
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.unsafe.types.UTF8String
+
+class FiltersV2Suite extends SparkFunSuite {
+
+  test("nested columns") {
+val filter1 = new EqualTo(ref("a", "B"), LiteralValue(1, IntegerType))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a.B"))
+assert(filter1.describe.equals("a.B = 1"))
+
+val filter2 = new EqualTo(ref("a", "b.c"), LiteralValue(1, IntegerType))
+assert(filter2.references.map(_.describe()).toSeq == Seq("a.`b.c`"))
+assert(filter2.describe.equals("a.`b.c` = 1"))
+
+val filter3 = new EqualTo(ref("`a`.b", "c"), LiteralValue(1, IntegerType))
+assert(filter3.references.map(_.describe()).toSeq == Seq("```a``.b`.c"))
+assert(filter3.describe.equals("```a``.b`.c = 1"))
+  }
+
+  test("EqualTo") {
+val filter1 = new EqualTo(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new EqualTo(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a = 1"))
+  }
+
+  test("EqualNullSafe") {
+val filter1 = new EqualNullSafe(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new EqualNullSafe(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a <=> 1"))
+  }
+
+  test("GreaterThan") {
+val filter1 = new GreaterThan(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new GreaterThan(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a > 1"))
+  }
+
+  test("GreaterThanOrEqual") {
+val filter1 = new GreaterThanOrEqual(ref("a"), LiteralValue(1, 
IntegerType))
+val filter2 = new GreaterThanOrEqual(ref("a"), LiteralValue(1, 
IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a >= 1"))
+  }
+
+  test("LessThan") {
+val filter1 = new LessThan(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new LessThan(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a < 1"))
+  }
+
+  test("LessThanOrEqual") {
+val filter1 = new LessThanOrEqual(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new LessThanOrEqual(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a <= 1"))
+  }
+
+  test("In") {
+val filter1 = new In(ref("a"),
+  Array(LiteralValue(1, IntegerType), LiteralValue(2, IntegerType),
+LiteralValue(3, IntegerType), LiteralValue(4, IntegerType)))
+val filter2 = new In(ref("a"),
+  Array(LiteralValue(1, IntegerType), LiteralValue(2, IntegerType),
+LiteralValue(3, IntegerType), LiteralValue(4, IntegerType)))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a IN (1, 2, 3, 4)"))
+  }
+
+  test("IsNull") {
+val filter1 = new IsNull(ref("a"))
+val filter2 = new IsNull(ref("a"))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+

[GitHub] [spark] cloud-fan commented on a change in pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


cloud-fan commented on a change in pull request #33803:
URL: https://github.com/apache/spark/pull/33803#discussion_r704045418



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2FiltersSuite.scala
##
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.connector.expressions.{FieldReference, 
LiteralValue}
+import org.apache.spark.sql.connector.expressions.filter._
+import org.apache.spark.sql.execution.datasources.v2.FiltersV2Suite.ref
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.unsafe.types.UTF8String
+
+class FiltersV2Suite extends SparkFunSuite {
+
+  test("nested columns") {
+val filter1 = new EqualTo(ref("a", "B"), LiteralValue(1, IntegerType))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a.B"))
+assert(filter1.describe.equals("a.B = 1"))
+
+val filter2 = new EqualTo(ref("a", "b.c"), LiteralValue(1, IntegerType))
+assert(filter2.references.map(_.describe()).toSeq == Seq("a.`b.c`"))
+assert(filter2.describe.equals("a.`b.c` = 1"))
+
+val filter3 = new EqualTo(ref("`a`.b", "c"), LiteralValue(1, IntegerType))
+assert(filter3.references.map(_.describe()).toSeq == Seq("```a``.b`.c"))
+assert(filter3.describe.equals("```a``.b`.c = 1"))
+  }
+
+  test("EqualTo") {
+val filter1 = new EqualTo(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new EqualTo(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a = 1"))
+  }
+
+  test("EqualNullSafe") {
+val filter1 = new EqualNullSafe(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new EqualNullSafe(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a <=> 1"))
+  }
+
+  test("GreaterThan") {
+val filter1 = new GreaterThan(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new GreaterThan(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a > 1"))
+  }
+
+  test("GreaterThanOrEqual") {
+val filter1 = new GreaterThanOrEqual(ref("a"), LiteralValue(1, 
IntegerType))
+val filter2 = new GreaterThanOrEqual(ref("a"), LiteralValue(1, 
IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a >= 1"))
+  }
+
+  test("LessThan") {
+val filter1 = new LessThan(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new LessThan(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a < 1"))
+  }
+
+  test("LessThanOrEqual") {
+val filter1 = new LessThanOrEqual(ref("a"), LiteralValue(1, IntegerType))
+val filter2 = new LessThanOrEqual(ref("a"), LiteralValue(1, IntegerType))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a <= 1"))
+  }
+
+  test("In") {
+val filter1 = new In(ref("a"),
+  Array(LiteralValue(1, IntegerType), LiteralValue(2, IntegerType),
+LiteralValue(3, IntegerType), LiteralValue(4, IntegerType)))
+val filter2 = new In(ref("a"),
+  Array(LiteralValue(1, IntegerType), LiteralValue(2, IntegerType),
+LiteralValue(3, IntegerType), LiteralValue(4, IntegerType)))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+assert(filter1.describe.equals("a IN (1, 2, 3, 4)"))
+  }
+
+  test("IsNull") {
+val filter1 = new IsNull(ref("a"))
+val filter2 = new IsNull(ref("a"))
+assert(filter1.equals(filter2))
+assert(filter1.references.map(_.describe()).toSeq == Seq("a"))
+

[GitHub] [spark] itholic commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r704045118



##
File path: python/pyspark/pandas/series.py
##
@@ -4475,6 +4477,146 @@ def replace(
 
 return self._with_new_scol(current)  # TODO: dtype?
 
+def combine(
+self,
+other: "Series",
+func: Callable,
+fill_value: Optional[Any] = None,
+return_type: Union[Union[AtomicType, str], ArrayType] = "string",
+) -> "Series":
+"""
+Combine the Series with a Series or scalar according to `func`.
+Combine the Series and `other` using `func` to perform elementwise
+selection for combined Series.
+`fill_value` is assumed when value is missing at some index
+from one of the two objects being combined.
+
+.. versionadded:: 3.3.0
+
+Parameters
+--
+other : Series or scalar
+The value(s) to be combined with the `Series`.
+func : function
+Function that takes two scalars as inputs and returns an element.
+fill_value : scalar, optional
+The value to assume when an index is missing from
+one Series or the other. The default specifies to use the
+appropriate NaN value for the underlying dtype of the Series.
+return_type : :class:`pyspark.sql.types.DataType` or str
+the return type of the output Series. The value can be either a
+:class:`pyspark.sql.types.DataType` object or a DDL-formatted type 
string.
+
+Returns
+---
+Series
+The result of combining the Series with the other object.
+
+See Also
+
+Series.combine_first : Combine Series values, choosing the calling
+Series' values first.
+
+Examples
+
+Consider 2 Datasets ``s1`` and ``s2`` containing
+highest clocked speeds of different birds.
+
+>>> from pyspark.pandas.config import set_option, reset_option
+>>> set_option("compute.ops_on_diff_frames", True)
+>>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+>>> s1
+falcon330.0
+eagle 160.0
+dtype: float64
+>>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+>>> s2
+falcon345.0
+eagle 200.0
+duck   30.0
+dtype: float64
+
+Now, to combine the two datasets and view the highest speeds
+of the birds across the two datasets
+
+>>> s1.combine(s2, max, return_type="double")
+duckNaN
+eagle 200.0
+falcon345.0
+dtype: float64
+
+In the previous example, the resulting value for duck is missing,
+because the maximum of a NaN and a float is a NaN.
+So, in the example, we set ``fill_value=0``,
+so the maximum value returned will be the value from some dataset.
+
+>>> s1.combine(s2, max, fill_value=0, return_type="double")
+duck   30.0
+eagle 200.0
+falcon345.0
+dtype: float64
+>>> reset_option("compute.ops_on_diff_frames")
+"""
+if not isinstance(other, Series) and not np.isscalar(other):
+raise TypeError("unsupported type: %s" % type(other))
+
+tmp_other_col = verify_temp_column_name(self._internal.spark_frame, 
"__tmp_other_col__")
+if np.isscalar(other):
+combined = DataFrame(
+InternalFrame(
+spark_frame=self._internal.spark_frame.select(
+*self._internal.spark_columns
+).withColumn(tmp_other_col, SF.lit(other)),
+index_spark_columns=self._internal.index_spark_columns,
+)
+)
+elif same_anchor(self, other):
+combined = self._psdf[self._column_label, other._column_label]
+elif fill_value is None:
+combined = combine_frames(self.to_frame(), other.to_frame())
+else:
+combined = self._combine_frame_with_fill_value(other, 
fill_value=fill_value)
+
+@pandas_udf(returnType=return_type)  # type: ignore
+def wrapped_func(x: pd.Series, y: pd.Series) -> pd.Series:
+return x.combine(y, func)
+
+internal = InternalFrame(
+spark_frame=combined._internal.spark_frame.select(
+*combined._internal.index_spark_columns,
+wrapped_func(*combined._internal.data_spark_columns),
+NATURAL_ORDER_COLUMN_NAME
+),
+index_spark_columns=combined._internal.index_spark_columns,
+column_labels=self._internal.column_labels if np.isscalar(other) 
else [None],
+column_label_names=[None],
+)

Review comment:
   Oh, yeah it's my mistake.
   
   I think we should get a 

[GitHub] [spark] dgd-contributor commented on a change in pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


dgd-contributor commented on a change in pull request #33911:
URL: https://github.com/apache/spark/pull/33911#discussion_r704017833



##
File path: python/pyspark/pandas/data_type_ops/boolean_ops.py
##
@@ -248,6 +249,30 @@ def and_func(left: Column, right: Any) -> Column:
 
 return column_op(and_func)(left, right)
 
+def __xor__(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
+_sanitize_list_like(right)
+if isinstance(right, IndexOpsMixin) and isinstance(right.dtype, 
extension_dtypes):
+return right.__xor__(left)
+elif (
+isinstance(right, int)
+or (hasattr(right, "dtype") and np.issubdtype(right.dtype, int))
+or isinstance(right, bool)
+or (hasattr(right, "dtype") and right.dtype == "bool")

Review comment:
   Thank you
   Yeah, they have same result. But it throw "Circular import" when I try to 
extract that  ```elif```  to ```data_type_ops/base.py```.  I do not understand 
```IndexOpsMixin```, I am appreciated if you could explain it and the better 
point. I deduplicated code. Could you take another look?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] rmcyang opened a new pull request #33932: [Spark-33781][SHUFFLE] Improve caching of MergeStatus on the executor side to save memory

2021-09-07 Thread GitBox


rmcyang opened a new pull request #33932:
URL: https://github.com/apache/spark/pull/33932


   
   
   ### What changes were proposed in this pull request?
   
   This is one of the patches for SPARK-33235: Push-based Shuffle Improvement 
Tasks.
   At high level, in `MapOutputTrackerWorker`, if serialized `MergeStatuse` 
array size is larger than 
threshold(`spark.shuffle.push.mergeResult.minSizeForReducedCache`), cache the 
much more compact serialized bytes instead and only cache the deserialized 
`MergeStatus` objects that are needed (within `startPartitionId` until 
`endPartitionId`). Then deserialize the `MergeStatus` array from the cached 
serialized bytes again.
   
   
   ### Why are the changes needed?
   
   For large shuffles with 10s or 100s of thousands of shuffle partitions, 
caching the entire deserialized and decompressed MergeStatus array on the 
executor side, while perhaps only 0.1% of them are going to be used by the 
tasks running in this executor is a huge waste of memory.
   This change helps save memory as well as helps with reducing GC pressure on 
executor side.
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes. This PR introduces a client-side config for push-based 
shuffle(`spark.shuffle.push.mergeResult.minSizeForReducedCache`). If push-based 
shuffle is turned-off then the users will not see any change.
   
   
   ### How was this patch tested?
   
   Added unit test.
   Verified effectiveness with jobs that would fail due to GC issue otherwise.
   
   Ran the benchmark using GitHub Actions and did not observe any performance 
penalties. The results are attached in this PR:
   ```
   core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt
   core/benchmarks/MapStatusesSerDeserBenchmark-results.txt
   ```
   
   Lead-authored-by: Min Shen ms...@linkedin.com
   Co-authored-by: Chandni Singh chsi...@linkedin.com
   Co-authored-by: Minchu Yang miny...@linkedin.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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dongjoon-hyun commented on pull request #33921: [SPARK-36677][SQL] NestedColumnAliasing should not push down aggregate functions into projections

2021-09-07 Thread GitBox


dongjoon-hyun commented on pull request #33921:
URL: https://github.com/apache/spark/pull/33921#issuecomment-914911930


   +1, LGTM. Thank you, @vicennial , @HyukjinKwon , @hvanhovell , @viirya .


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dongjoon-hyun commented on pull request #33903: [SPARK-36656][SQL] CollapseProject should not collapse correlated scalar subqueries

2021-09-07 Thread GitBox


dongjoon-hyun commented on pull request #33903:
URL: https://github.com/apache/spark/pull/33903#issuecomment-914911371


   Thank you for the PR description updates, @allisonwang-db . 


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dongjoon-hyun commented on pull request #33926: [SPARK-36684][SQL][TESTS] Add Jackson test dependencies to `sql/core` module at `hadoop-2.7` profile

2021-09-07 Thread GitBox


dongjoon-hyun commented on pull request #33926:
URL: https://github.com/apache/spark/pull/33926#issuecomment-914907058


   Merged to master/3.2.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dongjoon-hyun closed pull request #33926: [SPARK-36684][SQL][TESTS] Add Jackson test dependencies to `sql/core` module at `hadoop-2.7` profile

2021-09-07 Thread GitBox


dongjoon-hyun closed pull request #33926:
URL: https://github.com/apache/spark/pull/33926


   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914903936


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143065/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA removed a comment on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


SparkQA removed a comment on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914903508


   **[Test build #143065 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143065/testReport)**
 for PR 33911 at commit 
[`032bf4b`](https://github.com/apache/spark/commit/032bf4bc1d826b699a63f84cd117e5215fb37355).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914903936


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143065/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


SparkQA commented on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914903911


   **[Test build #143065 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143065/testReport)**
 for PR 33911 at commit 
[`032bf4b`](https://github.com/apache/spark/commit/032bf4bc1d826b699a63f84cd117e5215fb37355).
* This patch **fails Python style tests**.
* This patch merges cleanly.
* This patch adds no public classes.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


SparkQA commented on pull request #33911:
URL: https://github.com/apache/spark/pull/33911#issuecomment-914903508


   **[Test build #143065 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143065/testReport)**
 for PR 33911 at commit 
[`032bf4b`](https://github.com/apache/spark/commit/032bf4bc1d826b699a63f84cd117e5215fb37355).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


SparkQA commented on pull request #33931:
URL: https://github.com/apache/spark/pull/33931#issuecomment-914903403


   **[Test build #143064 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143064/testReport)**
 for PR 33931 at commit 
[`de6d1fd`](https://github.com/apache/spark/commit/de6d1fdc7e7769c0eaa136e7df4fda24e0eda5e1).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dgd-contributor commented on a change in pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


dgd-contributor commented on a change in pull request #33911:
URL: https://github.com/apache/spark/pull/33911#discussion_r704018411



##
File path: python/pyspark/pandas/base.py
##
@@ -422,12 +422,18 @@ def __and__(self, other: Any) -> SeriesOrIndex:
 def __or__(self, other: Any) -> SeriesOrIndex:
 return self._dtype_op.__or__(self, other)
 
+def __xor__(self, other: Any) -> SeriesOrIndex:

Review comment:
   updated, could you take another look please?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dgd-contributor commented on a change in pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


dgd-contributor commented on a change in pull request #33911:
URL: https://github.com/apache/spark/pull/33911#discussion_r704017833



##
File path: python/pyspark/pandas/data_type_ops/boolean_ops.py
##
@@ -248,6 +249,30 @@ def and_func(left: Column, right: Any) -> Column:
 
 return column_op(and_func)(left, right)
 
+def __xor__(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
+_sanitize_list_like(right)
+if isinstance(right, IndexOpsMixin) and isinstance(right.dtype, 
extension_dtypes):
+return right.__xor__(left)
+elif (
+isinstance(right, int)
+or (hasattr(right, "dtype") and np.issubdtype(right.dtype, int))
+or isinstance(right, bool)
+or (hasattr(right, "dtype") and right.dtype == "bool")

Review comment:
   Thank you
   Yeah, they have same result. But it throw "Circular import" when I try to 
extract that  ```elif```  to ```data_type_ops/base.py```.  I do not understand 
```IndexOpsMinx```, I am appreciated if you could explain it and the better 
point. I deduplicated code. Could you take another look?

##
File path: python/pyspark/pandas/data_type_ops/num_ops.py
##
@@ -181,6 +181,39 @@ class IntegralOps(NumericOps):
 LongType, IntegerType, ByteType and ShortType.
 """
 
+def __xor__(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
+_sanitize_list_like(right)
+
+right_is_boolean = (
+True
+if isinstance(right, bool) or (hasattr(right, "dtype") and 
right.dtype == "bool")
+else False
+)
+
+if isinstance(right, IndexOpsMixin) and isinstance(right.dtype, 
extension_dtypes):
+return right.__xor__(left)
+elif (

Review comment:
   thanks, deduplicated.




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic opened a new pull request #33931: [SPARK-36689][PYTHON] Cleanup the deprecated APIs and raise proper warning message.

2021-09-07 Thread GitBox


itholic opened a new pull request #33931:
URL: https://github.com/apache/spark/pull/33931


   ### What changes were proposed in this pull request?
   
   This PR proposes cleanup the deprecated APIs in `missing/*.py`, and raise 
proper warning message for the deprecated APIs such as pandas does.
   
   ### Why are the changes needed?
   
   We should follow the deprecation of APIs of latest pandas.
   
   ### Does this PR introduce _any_ user-facing change?
   
   Now the some APIs raise proper alternative message for deprecated functions 
such as pandas does.
   
   ### How was this patch tested?
   
   Ran `dev/lint-python` and manually check the pandas API documents one by one.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] cloud-fan closed pull request #33922: [SPARK-35803][SQL] Support DataSource V2 CreateTempViewUsing

2021-09-07 Thread GitBox


cloud-fan closed pull request #33922:
URL: https://github.com/apache/spark/pull/33922


   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] cloud-fan commented on pull request #33922: [SPARK-35803][SQL] Support DataSource V2 CreateTempViewUsing

2021-09-07 Thread GitBox


cloud-fan commented on pull request #33922:
URL: https://github.com/apache/spark/pull/33922#issuecomment-914899179


   thanks, merging to master!


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] gengliangwang commented on a change in pull request #33919: [SPARK-36674][SQL] Support ILIKE - case insensitive LIKE

2021-09-07 Thread GitBox


gengliangwang commented on a change in pull request #33919:
URL: https://github.com/apache/spark/pull/33919#discussion_r704000135



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -121,70 +187,73 @@ abstract class StringRegexExpression extends 
BinaryExpression
   since = "1.0.0",
   group = "predicate_funcs")
 // scalastyle:on line.contains.tab
-case class Like(left: Expression, right: Expression, escapeChar: Char)
-  extends StringRegexExpression {
+case class Like(left: Expression, right: Expression, escapeChar: Char) extends 
LikeBase {
 
   def this(left: Expression, right: Expression) = this(left, right, '\\')
 
-  override def escape(v: String): String = StringUtils.escapeLikeRegex(v, 
escapeChar)
+  override protected def likeName: String = "LIKE"
 
-  override def matches(regex: Pattern, str: String): Boolean = 
regex.matcher(str).matches()
+  override protected def withNewChildrenInternal(newLeft: Expression, 
newRight: Expression): Like =
+copy(left = newLeft, right = newRight)
+}
 
-  final override val nodePatterns: Seq[TreePattern] = Seq(LIKE_FAMLIY)
+// scalastyle:off line.contains.tab
+/**
+ * Simple RegEx case-insensitive pattern matching function
+ */
+@ExpressionDescription(
+  usage = "str _FUNC_ pattern[ ESCAPE escape] - Returns true if str matches 
`pattern` with " +
+"`escape` case-insensitively, null if any arguments are null, false 
otherwise.",
+  arguments = """
+Arguments:
+  * str - a string expression
+  * pattern - a string expression. The pattern is a string which is 
matched literally and
+  case-insensitively, with exception to the following special symbols:
 
-  override def toString: String = escapeChar match {
-case '\\' => s"$left LIKE $right"
-case c => s"$left LIKE $right ESCAPE '$c'"
-  }
+  _ matches any one character in the input (similar to . in posix 
regular expressions)
 
-  override def sql: String = s"${left.sql} 
${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}"
+  % matches zero or more characters in the input (similar to .* in 
posix regular
+  expressions)
 
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
-val patternClass = classOf[Pattern].getName
-val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
+  Since Spark 2.0, string literals are unescaped in our SQL parser. 
For example, in order
+  to match "\abc", the pattern should be "\\abc".
 
-if (right.foldable) {
-  val rVal = right.eval()
-  if (rVal != null) {
-val regexStr =
-  
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-val pattern = ctx.addMutableState(patternClass, "patternLike",
-  v => s"""$v = $patternClass.compile("$regexStr");""")
+  When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, 
it falls back
+  to Spark 1.6 behavior regarding string literal parsing. For example, 
if the config is
+  enabled, the pattern to match "\abc" should be "\abc".
+  * escape - an character added since Spark 3.0. The default escape 
character is the '\'.
+  If an escape character precedes a special symbol or another escape 
character, the
+  following character is matched literally. It is invalid to escape 
any other character.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_('Spark', '_Park');
+  true
+  > SET spark.sql.parser.escapedStringLiterals=true;
+  spark.sql.parser.escapedStringLiterals   true
+  > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\users%';
+  true
+  > SET spark.sql.parser.escapedStringLiterals=false;
+  spark.sql.parser.escapedStringLiterals   false
+  > SELECT '%SystemDrive%\\USERS\\John' _FUNC_ '\%SystemDrive\%Users%';
+  true
+  > SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SYSTEMDrive/%//Users%' 
ESCAPE '/';

Review comment:
   We should have a test case for ESCAPE char as well.




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] gengliangwang commented on a change in pull request #33919: [SPARK-36674][SQL] Support ILIKE - case insensitive LIKE

2021-09-07 Thread GitBox


gengliangwang commented on a change in pull request #33919:
URL: https://github.com/apache/spark/pull/33919#discussion_r703999493



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
##
@@ -121,70 +187,73 @@ abstract class StringRegexExpression extends 
BinaryExpression
   since = "1.0.0",
   group = "predicate_funcs")
 // scalastyle:on line.contains.tab
-case class Like(left: Expression, right: Expression, escapeChar: Char)
-  extends StringRegexExpression {
+case class Like(left: Expression, right: Expression, escapeChar: Char) extends 
LikeBase {
 
   def this(left: Expression, right: Expression) = this(left, right, '\\')
 
-  override def escape(v: String): String = StringUtils.escapeLikeRegex(v, 
escapeChar)
+  override protected def likeName: String = "LIKE"
 
-  override def matches(regex: Pattern, str: String): Boolean = 
regex.matcher(str).matches()
+  override protected def withNewChildrenInternal(newLeft: Expression, 
newRight: Expression): Like =
+copy(left = newLeft, right = newRight)
+}
 
-  final override val nodePatterns: Seq[TreePattern] = Seq(LIKE_FAMLIY)
+// scalastyle:off line.contains.tab
+/**
+ * Simple RegEx case-insensitive pattern matching function
+ */
+@ExpressionDescription(
+  usage = "str _FUNC_ pattern[ ESCAPE escape] - Returns true if str matches 
`pattern` with " +
+"`escape` case-insensitively, null if any arguments are null, false 
otherwise.",
+  arguments = """
+Arguments:
+  * str - a string expression
+  * pattern - a string expression. The pattern is a string which is 
matched literally and
+  case-insensitively, with exception to the following special symbols:
 
-  override def toString: String = escapeChar match {
-case '\\' => s"$left LIKE $right"
-case c => s"$left LIKE $right ESCAPE '$c'"
-  }
+  _ matches any one character in the input (similar to . in posix 
regular expressions)
 
-  override def sql: String = s"${left.sql} 
${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}"
+  % matches zero or more characters in the input (similar to .* in 
posix regular
+  expressions)
 
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
-val patternClass = classOf[Pattern].getName
-val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + 
".escapeLikeRegex"
+  Since Spark 2.0, string literals are unescaped in our SQL parser. 
For example, in order
+  to match "\abc", the pattern should be "\\abc".
 
-if (right.foldable) {
-  val rVal = right.eval()
-  if (rVal != null) {
-val regexStr =
-  
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
-val pattern = ctx.addMutableState(patternClass, "patternLike",
-  v => s"""$v = $patternClass.compile("$regexStr");""")
+  When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, 
it falls back
+  to Spark 1.6 behavior regarding string literal parsing. For example, 
if the config is
+  enabled, the pattern to match "\abc" should be "\abc".
+  * escape - an character added since Spark 3.0. The default escape 
character is the '\'.
+  If an escape character precedes a special symbol or another escape 
character, the
+  following character is matched literally. It is invalid to escape 
any other character.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_('Spark', '_Park');
+  true
+  > SET spark.sql.parser.escapedStringLiterals=true;

Review comment:
   We should have test case for enable/disable the conf




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] yaooqinn commented on a change in pull request #33888: [SPARK-36634][SQL] Support access and read parquet file by column ordinal

2021-09-07 Thread GitBox


yaooqinn commented on a change in pull request #33888:
URL: https://github.com/apache/spark/pull/33888#discussion_r703998755



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
##
@@ -901,6 +901,91 @@ abstract class ParquetQuerySuite extends QueryTest with 
ParquetTest with SharedS
   }
 }
   }
+
+  test("SPARK-36634: Support access and read parquet file by column index") {
+withTempDir { dir =>
+  val loc = s"file:///$dir/t"
+
+  withTable("t1", "t2", "t3") {
+sql(s"create table t1 (my_id int, my_name string) using parquet 
location '$loc'")
+sql(s"create table t2 (myid int, myName string) using parquet location 
'$loc'")
+sql("insert into t1 select 1, 'apache'")
+sql("insert into t2 select 2, 'software'")
+sql("insert into t2 select 3, 'foundation'")
+sql(s"create table t3 (myid int, myname string, myage int) using 
parquet location '$loc'")
+
+withSQLConf((SQLConf.PARQUET_ACCESS_BY_ORDINAL.key, "false")) {
+  checkAnswer(sql("select my_id from t1"), Seq(Row(1), Row(null), 
Row(null)))
+  checkAnswer(sql("select my_id, my_name from t1"),
+Seq(Row(1, "apache"), Row(null, null), Row(null, null)))
+  assert(sql("select my_id, my_name from t1 where my_id=2").isEmpty)
+  checkAnswer(sql("select myid, myname, myage from t3"),
+Seq(Row(2, "software", null),
+  Row(3, "foundation", null),
+  Row(null, null, null)))
+}
+
+sql("insert into t3 select 4, 'spark', 11")
+
+withAllParquetReaders {
+  withSQLConf((SQLConf.PARQUET_ACCESS_BY_ORDINAL.key, "true")) {
+checkAnswer(sql("select my_id from t1"), Seq(Row(1), Row(2), 
Row(3)))
+val e1 = {
+  intercept[SparkException](sql("select my_name from 
t1").collect())
+}
+assert(e1.getCause.getMessage.contains("Parquet column cannot be 
converted in"))

Review comment:
   I guess they are not conflicting. We are using idx-0 to get data from 
`file 1(my_id)` and  `file 2(myid)` and feed it to my_name. Here the idx is not 
the final output set but the original input set of raw data




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33922: [SPARK-35803][SQL] Support DataSource V2 CreateTempViewUsing

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33922:
URL: https://github.com/apache/spark/pull/33922#issuecomment-914707548






-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33922: [SPARK-35803][SQL] Support DataSource V2 CreateTempViewUsing

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33922:
URL: https://github.com/apache/spark/pull/33922#issuecomment-914881229


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143061/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA removed a comment on pull request #33922: [SPARK-35803][SQL] Support DataSource V2 CreateTempViewUsing

2021-09-07 Thread GitBox


SparkQA removed a comment on pull request #33922:
URL: https://github.com/apache/spark/pull/33922#issuecomment-914669562


   **[Test build #143061 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143061/testReport)**
 for PR 33922 at commit 
[`098c485`](https://github.com/apache/spark/commit/098c485d2df95d445d87a698b0ec754d1fa6497c).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] cloud-fan commented on pull request #33927: [SPARK-34952][SQL][FOLLOWUP] Change column type to be NamedReference

2021-09-07 Thread GitBox


cloud-fan commented on pull request #33927:
URL: https://github.com/apache/spark/pull/33927#issuecomment-914866940


   > "Fix problem" sounds too general. :)
   
   yea, should be something like `FieldReference is a private class`


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33922: [SPARK-35803][SQL] Support DataSource V2 CreateTempViewUsing

2021-09-07 Thread GitBox


SparkQA commented on pull request #33922:
URL: https://github.com/apache/spark/pull/33922#issuecomment-914865788


   **[Test build #143061 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143061/testReport)**
 for PR 33922 at commit 
[`098c485`](https://github.com/apache/spark/commit/098c485d2df95d445d87a698b0ec754d1fa6497c).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds no public classes.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] sleep1661 commented on a change in pull request #33872: [SPARK-36575][CORE] Should ignore task finished event if its task set is gone in TaskSchedulerImpl.handleSuccessfulTask

2021-09-07 Thread GitBox


sleep1661 commented on a change in pull request #33872:
URL: https://github.com/apache/spark/pull/33872#discussion_r703986125



##
File path: 
core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
##
@@ -1995,6 +2000,61 @@ class TaskSchedulerImplSuite extends SparkFunSuite with 
LocalSparkContext with B
 assert(!normalTSM.runningTasksSet.contains(taskId))
   }
 
+  test("SPARK-36575: Executor lost cause task hang") {
+val taskScheduler = setupScheduler()
+
+val resultGetter = new TaskResultGetter(sc.env, taskScheduler) {
+  override protected val getTaskResultExecutor: ExecutorService =
+ThreadUtils.newDaemonFixedThreadPool(1, "task-result-getter")
+  def taskResultExecutor() : ExecutorService = getTaskResultExecutor
+}
+taskScheduler.taskResultGetter = resultGetter
+
+val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 1),
+  new WorkerOffer("executor1", "host1", 1))
+val task1 = new ShuffleMapTask(1, 0, null, new Partition {
+  override def index: Int = 0
+}, Seq(TaskLocation("host0", "executor0")), new Properties, null)
+
+val task2 = new ShuffleMapTask(1, 0, null, new Partition {
+  override def index: Int = 0
+}, Seq(TaskLocation("host1", "executor1")), new Properties, null)
+
+val taskSet = new TaskSet(Array(task1, task2), 0, 0, 0, null, 0)
+
+taskScheduler.submitTasks(taskSet)
+val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+assert(2 === taskDescriptions.length)
+
+val ser = sc.env.serializer.newInstance()
+val directResult = new DirectTaskResult[Int](ser.serialize(1), Seq(), 
Array.empty)
+val resultBytes = ser.serialize(directResult)
+
+// make getTaskResultExecutor busy
+import scala.language.reflectiveCalls
+resultGetter.taskResultExecutor().submit( new Runnable {
+  override def run(): Unit = Thread.sleep(100)
+})
+
+// task1 finished
+taskScheduler.statusUpdate(
+  tid = taskDescriptions(0).taskId,
+  state = TaskState.FINISHED,
+  serializedData = resultBytes
+)
+
+// mark executor heartbeat timed out
+taskScheduler.executorLost(taskDescriptions(0).executorId, 
ExecutorProcessLost("Executor " +
+  "heartbeat timed out"))
+
+// Wait a while until all events are processed
+Thread.sleep(100)

Review comment:
   Yes. But it cause` TaskSetManager.successful` and 
`TaskSetManager.tasksSuccessful` wrong result in Matser, I think it's better to 
fix it. 




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] viirya commented on pull request #33927: [SPARK-34952][SQL][FOLLOWUP] Change column type to be NamedReference

2021-09-07 Thread GitBox


viirya commented on pull request #33927:
URL: https://github.com/apache/spark/pull/33927#issuecomment-914863184


   "Fix problem" sounds too general. :)
   
   @huaxingao Maybe add a few words there?


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] yutoacts commented on a change in pull request #33925: [SPARK-36688][R] Add cot as an R function

2021-09-07 Thread GitBox


yutoacts commented on a change in pull request #33925:
URL: https://github.com/apache/spark/pull/33925#discussion_r703985509



##
File path: R/pkg/R/functions.R
##
@@ -870,6 +870,19 @@ setMethod("cosh",
 column(jc)
   })
 
+#' @details
+#' \code{cot}: Returns the cotangent of the given value.
+#'
+#' @rdname column_math_functions
+#' @aliases cot cot,Column-method
+#' @note cot since 3.3.0
+setMethod("cot",

Review comment:
   Thanks, just added. 
   So, if I add a new function, should I make a JIRA ticket and PR separately 
just for R implementation?




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] cloud-fan commented on pull request #33924: [SPARK-36682][CORE][TEST] Add Hadoop sequence file test for different Hadoop codecs

2021-09-07 Thread GitBox


cloud-fan commented on pull request #33924:
URL: https://github.com/apache/spark/pull/33924#issuecomment-914861488


   late LGTM


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA removed a comment on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


SparkQA removed a comment on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914726982


   **[Test build #143063 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143063/testReport)**
 for PR 33930 at commit 
[`d425982`](https://github.com/apache/spark/commit/d425982410ecda5b8d26eb8178f33820caefd586).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914852940


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143063/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914852940


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143063/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


SparkQA commented on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914852515


   **[Test build #143063 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143063/testReport)**
 for PR 33930 at commit 
[`d425982`](https://github.com/apache/spark/commit/d425982410ecda5b8d26eb8178f33820caefd586).
* This patch **fails Spark unit tests**.
* This patch merges cleanly.
* This patch adds no public classes.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33803:
URL: https://github.com/apache/spark/pull/33803#issuecomment-914852110


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143060/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33803:
URL: https://github.com/apache/spark/pull/33803#issuecomment-914852110


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/143060/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA removed a comment on pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


SparkQA removed a comment on pull request #33803:
URL: https://github.com/apache/spark/pull/33803#issuecomment-914647983


   **[Test build #143060 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143060/testReport)**
 for PR 33803 at commit 
[`8c6f18c`](https://github.com/apache/spark/commit/8c6f18c9bd83c331db239c7541a93f2a3847895e).


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33803: [SPARK-36556][SQL] Add DSV2 filters

2021-09-07 Thread GitBox


SparkQA commented on pull request #33803:
URL: https://github.com/apache/spark/pull/33803#issuecomment-914850617


   **[Test build #143060 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/143060/testReport)**
 for PR 33803 at commit 
[`8c6f18c`](https://github.com/apache/spark/commit/8c6f18c9bd83c331db239c7541a93f2a3847895e).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds no public classes.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dgd-contributor commented on a change in pull request #33864: [SPARK-36296][SQL] Refactor seventh set of 20 in QueryExecutionErrors to use error classes

2021-09-07 Thread GitBox


dgd-contributor commented on a change in pull request #33864:
URL: https://github.com/apache/spark/pull/33864#discussion_r703973612



##
File path: core/src/main/resources/error/error-classes.json
##
@@ -3,9 +3,29 @@
 "message" : [ "Field name %s is ambiguous and has %s matching fields in 
the struct." ],
 "sqlState" : "42000"
   },
+  "CANNOT_GET_JDBC_TYPE" : {
+"message" : [ "Can't get JDBC type for %s" ],
+"sqlState" : "42000"
+  },
+  "CANNOT_READ_FOOTER_FOR_FILE" : {
+"message" : [ "Could not read footer for file: %s" ],
+"sqlState" : "42000"
+  },
+  "CANNOT_TRANSLATE_NON_NULL_VALUE_FOR_FIELD" : {
+"message" : [ "Can't translate non-null value for field %s" ],
+"sqlState" : "0A000"
+  },
   "CONCURRENT_QUERY_ERROR" : {
 "message" : [ "Another instance of this query was just started by a 
concurrent session." ]
   },
+  "DATA_TYPE_UNSUPPORTED_YET" : {
+"message" : [ "%s is not supported yet." ],
+"sqlState" : "0A000"
+  },
+  "DDL_UNSUPPORTED_TEMPORARILY" : {

Review comment:
   Thanks for your reviewing. Should I remove "temporarily" in the 
function's name, as well as in its message? And should I rename that function 
to "UNSUPPORTED_DDL"? @karenfeng 




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] dgd-contributor commented on a change in pull request #33929: [SPARK-36618] Support dropping rows of a single-indexed DataFrame

2021-09-07 Thread GitBox


dgd-contributor commented on a change in pull request #33929:
URL: https://github.com/apache/spark/pull/33929#discussion_r703961645



##
File path: python/pyspark/pandas/tests/test_dataframe.py
##
@@ -1287,8 +1278,42 @@ def test_drop(self):
 pdf = pd.DataFrame({10: [1, 2], 20: [3, 4], 30: [5, 6]}, 
index=np.random.rand(2))
 psdf = ps.from_pandas(pdf)
 
-self.assert_eq(psdf.drop(10), pdf.drop(10, axis=1))
-self.assert_eq(psdf.drop([20, 30]), pdf.drop([20, 30], axis=1))
+self.assert_eq(psdf.drop(10, axis=1), pdf.drop(10, axis=1))
+self.assert_eq(psdf.drop([20, 30], axis=1), pdf.drop([20, 30], axis=1))
+
+#
+# Drop rows by index
+#
+
+pdf = pd.DataFrame({"X": [1, 2, 3], "Y": [4, 5, 6], "Z": [7, 8, 9]}, 
index=["A", "B", "C"])
+psdf = ps.from_pandas(pdf)
+
+# Given labels (and axis = 0)
+self.assert_eq(psdf.drop(labels="A", axis=0), pdf.drop(labels="A", 
axis=0))
+self.assert_eq(psdf.drop(labels="A"), pdf.drop(labels="A"))
+self.assert_eq(psdf.drop(labels=["A", "C"], axis=0), 
pdf.drop(labels=["A", "C"], axis=0))
+self.assert_eq(
+psdf.drop(labels=["A", "B", "C"], axis=0), pdf.drop(labels=["A", 
"B", "C"], axis=0)
+)
+
+# Given index
+self.assert_eq(psdf.drop(index="A"), pdf.drop(index="A"))
+self.assert_eq(psdf.drop(index=["A", "C"]), pdf.drop(index=["A", "C"]))
+self.assert_eq(psdf.drop(index=["A", "B", "C"]), pdf.drop(index=["A", 
"B", "C"]))
+
+# Non-string names
+pdf.index = [10, 20, 30]
+psdf = ps.from_pandas(pdf)
+self.assert_eq(psdf.drop(labels=10, axis=0), pdf.drop(labels=10, 
axis=0))
+self.assert_eq(psdf.drop(labels=[10, 30], axis=0), 
pdf.drop(labels=[10, 30], axis=0))
+self.assert_eq(
+psdf.drop(labels=[10, 20, 30], axis=0), pdf.drop(labels=[10, 20, 
30], axis=0)
+)
+
+# MultiIndex
+pdf.index = pd.MultiIndex.from_tuples([("a", "x"), ("b", "y"), ("c", 
"z")])
+psdf = ps.from_pandas(pdf)
+self.assertRaises(NotImplementedError, lambda: psdf.drop(labels=[("a", 
"x")]))

Review comment:
   Should we support drop columns and row in a single time ? Like
   
   ```python
   >>> pdf = pd.DataFrame({"X": [1, 2, 3], "Y": [4, 5, 6], "Z": [7, 8, 9]}, 
index=["A", "B", "C"])
   >>> pdf.drop(columns="X", index="A")
  Y  Z
   B  5  8
   C  6  9
   ```




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] WeichenXu123 commented on pull request #33853: [SPARK-36642][SQL] Add df.withMetadata: a syntax suger to update the metadata of a dataframe

2021-09-07 Thread GitBox


WeichenXu123 commented on pull request #33853:
URL: https://github.com/apache/spark/pull/33853#issuecomment-914792425


   merged to master


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] WeichenXu123 closed pull request #33853: [SPARK-36642][SQL] Add df.withMetadata: a syntax suger to update the metadata of a dataframe

2021-09-07 Thread GitBox


WeichenXu123 closed pull request #33853:
URL: https://github.com/apache/spark/pull/33853


   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] itholic commented on a change in pull request #33911: [SPARK-36653][PYTHON] implement Series.__xor__ and Series.__rxor__

2021-09-07 Thread GitBox


itholic commented on a change in pull request #33911:
URL: https://github.com/apache/spark/pull/33911#discussion_r703957761



##
File path: python/pyspark/pandas/base.py
##
@@ -422,12 +422,18 @@ def __and__(self, other: Any) -> SeriesOrIndex:
 def __or__(self, other: Any) -> SeriesOrIndex:
 return self._dtype_op.__or__(self, other)
 
+def __xor__(self, other: Any) -> SeriesOrIndex:

Review comment:
   Oh, yeah. I missed it.
   
   Thanks for pointing out




-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33929: [SPARK-36618] Support dropping rows of a single-indexed DataFrame

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33929:
URL: https://github.com/apache/spark/pull/33929#issuecomment-914777320


   
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/47565/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins removed a comment on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


AmplabJenkins removed a comment on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914777321


   
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/47566/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914777321


   
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/47566/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] AmplabJenkins commented on pull request #33929: [SPARK-36618] Support dropping rows of a single-indexed DataFrame

2021-09-07 Thread GitBox


AmplabJenkins commented on pull request #33929:
URL: https://github.com/apache/spark/pull/33929#issuecomment-914777320


   
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/47565/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon closed pull request #33850: [SPARK-36531][SPARK-36515][PYTHON] Improve test coverage for data_type_ops/* and groupby.

2021-09-07 Thread GitBox


HyukjinKwon closed pull request #33850:
URL: https://github.com/apache/spark/pull/33850


   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon edited a comment on pull request #33850: [SPARK-36531][SPARK-36515][PYTHON] Improve test coverage for data_type_ops/* and groupby.

2021-09-07 Thread GitBox


HyukjinKwon edited a comment on pull request #33850:
URL: https://github.com/apache/spark/pull/33850#issuecomment-914771797


   Merged to master and branch-3.2.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on pull request #33850: [SPARK-36531][SPARK-36515][PYTHON] Improve test coverage for data_type_ops/* and groupby.

2021-09-07 Thread GitBox


HyukjinKwon commented on pull request #33850:
URL: https://github.com/apache/spark/pull/33850#issuecomment-914771797


   Merged to master.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


SparkQA commented on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914770209


   Kubernetes integration test status failure
   URL: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47566/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on pull request #33924: [SPARK-36682][CORE][TEST] Add Hadoop sequence file test for different Hadoop codecs

2021-09-07 Thread GitBox


HyukjinKwon commented on pull request #33924:
URL: https://github.com/apache/spark/pull/33924#issuecomment-914768778


   LGTM2


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] HyukjinKwon commented on pull request #33910: [SPARK-36666][SQL] Fix regression in AQEShuffleReadExec

2021-09-07 Thread GitBox


HyukjinKwon commented on pull request #33910:
URL: https://github.com/apache/spark/pull/33910#issuecomment-914767854


   Thanks LGTM2.
   
   BTW, it would have been nicer to fix the PR title .. otherwise every 
regression in AQE would have the same PR title with 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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] viirya closed pull request #33921: [SPARK-36677][SQL] NestedColumnAliasing should not push down aggregate functions into projections

2021-09-07 Thread GitBox


viirya closed pull request #33921:
URL: https://github.com/apache/spark/pull/33921


   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] viirya commented on pull request #33921: [SPARK-36677][SQL] NestedColumnAliasing should not push down aggregate functions into projections

2021-09-07 Thread GitBox


viirya commented on pull request #33921:
URL: https://github.com/apache/spark/pull/33921#issuecomment-914764690


   Thanks for your contribution! Merging to master/3.2.


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33930: [SPARK-36665][SQL] Add more Not operator simplifications

2021-09-07 Thread GitBox


SparkQA commented on pull request #33930:
URL: https://github.com/apache/spark/pull/33930#issuecomment-914760492


   Kubernetes integration test starting
   URL: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47566/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



[GitHub] [spark] SparkQA commented on pull request #33929: [SPARK-36618] Support dropping rows of a single-indexed DataFrame

2021-09-07 Thread GitBox


SparkQA commented on pull request #33929:
URL: https://github.com/apache/spark/pull/33929#issuecomment-914753817


   Kubernetes integration test status failure
   URL: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/47565/
   


-- 
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: reviews-unsubscr...@spark.apache.org

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



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



  1   2   3   4   5   >