[GitHub] [spark] jerryshao commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API

2019-07-16 Thread GitBox
jerryshao commented on a change in pull request #25007: 
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API 
URL: https://github.com/apache/spark/pull/25007#discussion_r304245623
 
 

 ##
 File path: 
core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java
 ##
 @@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle.api;
+
+import java.io.IOException;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * :: Experimental ::
+ * Indicates that partition writers can transfer bytes directly from input 
byte channels to
+ * output channels that stream data to the underlying shuffle partition 
storage medium.
+ * 
+ * This API is separated out for advanced users because it only needs to be 
used for
+ * specific low-level optimizations. The idea is that the returned channel can 
transfer bytes
+ * from the input file channel out to the backing storage system without 
copying data into
+ * memory.
+ * 
+ * Most shuffle plugin implementations should use {@link 
ShufflePartitionWriter} instead.
+ *
+ * @since 3.0.0
+ */
+@Experimental
+public interface SupportsTransferTo extends ShufflePartitionWriter {
 
 Review comment:
   I tend to agree with @vanzin , at least we could change to 
`TransferToSupport`. The mix-in semantics here is a little strange for Java, 
and actually we use it more like inheritance.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jerryshao commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API

2019-07-16 Thread GitBox
jerryshao commented on a change in pull request #25007: 
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API 
URL: https://github.com/apache/spark/pull/25007#discussion_r304243823
 
 

 ##
 File path: 
core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java
 ##
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle.api;
+
+import java.io.IOException;
+
+import org.apache.spark.annotation.Private;
+
+/**
+ * :: Experimental ::
+ * A top-level writer that returns child writers for persisting the output of 
a map task,
+ * and then commits all of the writes as one atomic operation.
+ *
+ * @since 3.0.0
+ */
+@Private
+public interface ShuffleMapOutputWriter {
+
+  /**
+   * Creates a writer that can open an output stream to persist bytes for a 
given chunk of
+   * a map task.
+   * 
+   * The chunk corresponds to bytes in a partition that all share the same 
reduce id, hence
+   * the given argument. This will not be called twice for the same partition 
identifier.
+   * The partition identifier will be in the range of precisely 0 (inclusive) 
to numPartitions
+   * (exclusive), where numPartitions was provided upon the creation of this 
map output writer via
+   * {@link ShuffleWriteSupport#createMapOutputWriter(int, int, long, int)}.
+   */
+  ShufflePartitionWriter getPartitionWriter(int partitionId) throws 
IOException;
+
+  /**
+   * Commits the writes done by all partition writers returned by all calls to 
this object's
+   * {@link #getPartitionWriter(int)}.
+   * 
+   * This should ensure that the writes conducted by this module's partition 
writers are
+   * available to downstream reduce tasks. If this method throws any 
exception, this module's
+   * {@link #abort(Throwable)} method will be invoked before propagating the 
exception.
+   * 
+   * This can also close any resources and clean up temporary state if 
necessary.
+   */
+  void commitAllPartitions() throws IOException;
 
 Review comment:
   @gczsjdy any reason to return `Optional`?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25177: [SPARK-28418][PYTHON][SQL] Wait for event process in 'test_query_execution_listener_on_collect'

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25177: [SPARK-28418][PYTHON][SQL] Wait for 
event process in 'test_query_execution_listener_on_collect'
URL: https://github.com/apache/spark/pull/25177#issuecomment-512126140
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] felixcheung commented on issue #24796: [SPARK-27900][CORE] Add uncaught exception handler to the driver

2019-07-16 Thread GitBox
felixcheung commented on issue #24796: [SPARK-27900][CORE] Add uncaught 
exception handler to the driver
URL: https://github.com/apache/spark/pull/24796#issuecomment-512126336
 
 
   Yay from me.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25177: [SPARK-28418][PYTHON][SQL] Wait for event process in 'test_query_execution_listener_on_collect'

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25177: [SPARK-28418][PYTHON][SQL] Wait for 
event process in 'test_query_execution_listener_on_collect'
URL: https://github.com/apache/spark/pull/25177#issuecomment-512126144
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12895/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25177: [SPARK-28418][PYTHON][SQL] Wait for event process in 'test_query_execution_listener_on_collect'

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25177: [SPARK-28418][PYTHON][SQL] 
Wait for event process in 'test_query_execution_listener_on_collect'
URL: https://github.com/apache/spark/pull/25177#issuecomment-512126144
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12895/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25177: [SPARK-28418][PYTHON][SQL] Wait for event process in 'test_query_execution_listener_on_collect'

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25177: [SPARK-28418][PYTHON][SQL] 
Wait for event process in 'test_query_execution_listener_on_collect'
URL: https://github.com/apache/spark/pull/25177#issuecomment-512126140
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25177: [SPARK-28418][PYTHON][SQL] Wait for event process in 'test_query_execution_listener_on_collect'

2019-07-16 Thread GitBox
SparkQA commented on issue #25177: [SPARK-28418][PYTHON][SQL] Wait for event 
process in 'test_query_execution_listener_on_collect'
URL: https://github.com/apache/spark/pull/25177#issuecomment-512126752
 
 
   **[Test build #107776 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107776/testReport)**
 for PR 25177 at commit 
[`74394ce`](https://github.com/apache/spark/commit/74394ce254c1cf53ebe6192bfb560c328857dec3).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] HyukjinKwon opened a new pull request #25177: [SPARK-28418][PYTHON][SQL] Wait for event process in 'test_query_execution_listener_on_collect'

2019-07-16 Thread GitBox
HyukjinKwon opened a new pull request #25177: [SPARK-28418][PYTHON][SQL] Wait 
for event process in 'test_query_execution_listener_on_collect'
URL: https://github.com/apache/spark/pull/25177
 
 
   ## What changes were proposed in this pull request?
   
   It fixes a flaky test:
   
   ```
   ERROR [0.164s]: test_query_execution_listener_on_collect 
(pyspark.sql.tests.test_dataframe.QueryExecutionListenerTests)
   --
   Traceback (most recent call last):
 File "/home/jenkins/python/pyspark/sql/tests/test_dataframe.py", line 758, 
in test_query_execution_listener_on_collect
   "The callback from the query execution listener should be called after 
'collect'")
   AssertionError: The callback from the query execution listener should be 
called after 'collect'
   ```
   
   Seems it can be failed because the event was somehow delayed but checked 
first.
   
   ## How was this patch tested?
   
   Manually.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] felixcheung commented on a change in pull request #24939: [SPARK-18569][ML][R] Support RFormula arithmetic, I() and spark functions

2019-07-16 Thread GitBox
felixcheung commented on a change in pull request #24939: [SPARK-18569][ML][R] 
Support RFormula arithmetic, I() and spark functions
URL: https://github.com/apache/spark/pull/24939#discussion_r304240770
 
 

 ##
 File path: mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala
 ##
 @@ -614,3 +652,80 @@ private object VectorAttributeRewriter extends 
MLReadable[VectorAttributeRewrite
 }
   }
 }
+
+/**
+ * Utility transformer for adding expressions to dataframe using `expr` spark 
function
+ *
+ * @param exprsToSelect set of string expressions to be added as a column to 
the dataframe.
+ *  The name of the columns will be identical to the 
expression
+ */
+private class ExprSelector(
 
 Review comment:
   if the pipeline model is persisted the "hidden" stage will show up, I think? 
it's good intention for the abstract but probably best not to add to the 
confusion there also since the code isn't too complicated. if sharing the logic 
is desired later we could refactor it but likely not as a private stage anyway.
   


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob Resolution in DoAs to use ProxyUser Credentials

2019-07-16 Thread GitBox
SparkQA commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob 
Resolution in DoAs to use ProxyUser Credentials
URL: https://github.com/apache/spark/pull/25176#issuecomment-512124636
 
 
   **[Test build #107775 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107775/testReport)**
 for PR 25176 at commit 
[`770a733`](https://github.com/apache/spark/commit/770a733bc66c8ac681f373ede94574dabc1d4cfe).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25176: [SPARK-28417][CORE] Wrap File Glob Resolution in DoAs to use ProxyUser Credentials

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25176: [SPARK-28417][CORE] Wrap File 
Glob Resolution in DoAs to use ProxyUser Credentials
URL: https://github.com/apache/spark/pull/25176#issuecomment-512124087
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12894/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25176: [SPARK-28417][CORE] Wrap File Glob Resolution in DoAs to use ProxyUser Credentials

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25176: [SPARK-28417][CORE] Wrap File 
Glob Resolution in DoAs to use ProxyUser Credentials
URL: https://github.com/apache/spark/pull/25176#issuecomment-512124081
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob Resolution in DoAs to use ProxyUser Credentials

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob 
Resolution in DoAs to use ProxyUser Credentials
URL: https://github.com/apache/spark/pull/25176#issuecomment-512124081
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob Resolution in DoAs to use ProxyUser Credentials

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob 
Resolution in DoAs to use ProxyUser Credentials
URL: https://github.com/apache/spark/pull/25176#issuecomment-512124087
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12894/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25176: [SPARK-28417][CORE] Wrap File Glob Resolution in DoAs to use ProxyUser Credentials

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25176: [SPARK-28417][CORE] Wrap File 
Glob Resolution in DoAs to use ProxyUser Credentials
URL: https://github.com/apache/spark/pull/25176#issuecomment-512006177
 
 
   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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] felixcheung commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob Resolution in DoAs to use ProxyUser Credentials

2019-07-16 Thread GitBox
felixcheung commented on issue #25176: [SPARK-28417][CORE] Wrap File Glob 
Resolution in DoAs to use ProxyUser Credentials
URL: https://github.com/apache/spark/pull/25176#issuecomment-512123665
 
 
   Jenkins, ok to test


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] HyukjinKwon commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
HyukjinKwon commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512122461
 
 
   Yup, that's fine with me too.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304214964
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 ##
 @@ -1223,6 +1223,12 @@ class AstBuilder(conf: SQLConf) extends 
SqlBaseBaseVisitor[AnyRef] with Logging
   case c: CreateNamedStruct => c.valExprs
   case other => Seq(other)
 }
+// Check the argument of boolean test is valid.
+def checkBooleanTestArgs(e: Expression): Unit = e.dataType match {
 
 Review comment:
   private not allowd here. It will makes a error:
   ```
   ESC[0m[ESC[31merrorESC[0m] 
ESC[0m/home/xitong/code/gengjiaan/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:1227:
 illegal start of statement (no modifiers allowed here)ESC[0m
   ESC[0m[ESC[31merrorESC[0m] ESC[0mprivate def checkBooleanTestArgs(e: 
Expression): Unit = e.dataType match {ESC[0m
   ```


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jerryshao commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API

2019-07-16 Thread GitBox
jerryshao commented on a change in pull request #25007: 
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API 
URL: https://github.com/apache/spark/pull/25007#discussion_r304237265
 
 

 ##
 File path: 
core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
 ##
 @@ -79,6 +83,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) 
extends ShuffleManager
*/
   private[this] val numMapsForShuffle = new ConcurrentHashMap[Int, Int]()
 
+  private lazy val shuffleExecutorComponents = 
loadShuffleExecutorComponents(conf)
+
   override val shuffleBlockResolver = new IndexShuffleBlockResolver(conf)
 
 Review comment:
   Yes, please add a TODO and create a JIRA to track 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304235680
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BooleanExpressionsSuite.scala
 ##
 @@ -0,0 +1,54 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.BooleanTest._
+import org.apache.spark.sql.types._
+
+class BooleanExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  val row0 = create_row(null)
+  val row1 = create_row(false)
+  val row2 = create_row(true)
+
+  test("istrue and isnottrue") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), TRUE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), TRUE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), TRUE), 
false, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
TRUE)), true, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), TRUE), 
true, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), TRUE)), 
false, row2)
+  }
+
+  test("isfalse and isnotfalse") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), FALSE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), FALSE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), FALSE), 
true, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
FALSE)), false, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), FALSE), 
false, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), 
FALSE)), true, row2)
+  }
+
+  test("isunknown and isnotunknown") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), UNKNOWN), 
true, row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), UNKNOWN)), 
false, row0)
+  }
+
+}
+
 
 Review comment:
   Scala style checks the empty line.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304235680
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BooleanExpressionsSuite.scala
 ##
 @@ -0,0 +1,54 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.BooleanTest._
+import org.apache.spark.sql.types._
+
+class BooleanExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  val row0 = create_row(null)
+  val row1 = create_row(false)
+  val row2 = create_row(true)
+
+  test("istrue and isnottrue") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), TRUE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), TRUE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), TRUE), 
false, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
TRUE)), true, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), TRUE), 
true, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), TRUE)), 
false, row2)
+  }
+
+  test("isfalse and isnotfalse") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), FALSE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), FALSE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), FALSE), 
true, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
FALSE)), false, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), FALSE), 
false, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), 
FALSE)), true, row2)
+  }
+
+  test("isunknown and isnotunknown") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), UNKNOWN), 
true, row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), UNKNOWN)), 
false, row0)
+  }
+
+}
+
 
 Review comment:
   Scala style check the empty line.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304235534
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BooleanExpressionsSuite.scala
 ##
 @@ -0,0 +1,54 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.BooleanTest._
+import org.apache.spark.sql.types._
+
+class BooleanExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  val row0 = create_row(null)
+  val row1 = create_row(false)
+  val row2 = create_row(true)
+
+  test("istrue and isnottrue") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), TRUE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), TRUE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), TRUE), 
false, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
TRUE)), true, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), TRUE), 
true, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), TRUE)), 
false, row2)
+  }
+
+  test("isfalse and isnotfalse") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), FALSE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), FALSE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), FALSE), 
true, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
FALSE)), false, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), FALSE), 
false, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), 
FALSE)), true, row2)
+  }
+
+  test("isunknown and isnotunknown") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), UNKNOWN), 
true, row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), UNKNOWN)), 
false, row0)
+  }
+
 
 Review comment:
   OK.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304235237
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BooleanExpressionsSuite.scala
 ##
 @@ -0,0 +1,54 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.BooleanTest._
+import org.apache.spark.sql.types._
+
+class BooleanExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  val row0 = create_row(null)
+  val row1 = create_row(false)
+  val row2 = create_row(true)
+
+  test("istrue and isnottrue") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), TRUE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), TRUE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), TRUE), 
false, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
TRUE)), true, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), TRUE), 
true, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), TRUE)), 
false, row2)
+  }
+
+  test("isfalse and isnotfalse") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), FALSE), false, 
row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), FALSE)), 
true, row0)
+checkEvaluation(BooleanTest(Literal.create(false, BooleanType), FALSE), 
true, row1)
+checkEvaluation(Not(BooleanTest(Literal.create(false, BooleanType), 
FALSE)), false, row1)
+checkEvaluation(BooleanTest(Literal.create(true, BooleanType), FALSE), 
false, row2)
+checkEvaluation(Not(BooleanTest(Literal.create(true, BooleanType), 
FALSE)), true, row2)
+  }
+
+  test("isunknown and isnotunknown") {
+checkEvaluation(BooleanTest(Literal.create(null, NullType), UNKNOWN), 
true, row0)
+checkEvaluation(Not(BooleanTest(Literal.create(null, NullType), UNKNOWN)), 
false, row0)
 
 Review comment:
   OK. I will add some.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] yaooqinn commented on issue #24947: [SPARK-28143][SQL] Expressions without proper constructors should throw AnalysisException

2019-07-16 Thread GitBox
yaooqinn commented on issue #24947: [SPARK-28143][SQL] Expressions without 
proper constructors should throw AnalysisException
URL: https://github.com/apache/spark/pull/24947#issuecomment-512117564
 
 
   @srowen I have updated the pr description and title as last comment 
required. If anything I misread, please correct me. Thanks very much.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] ueshin commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
ueshin commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with 
overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512115296
 
 
   I'm wondering what if a user explicitly specifies `overwrite=False`?
   E.g.,
   
   ```py
   df.write.mode("overwrite").insertInto("table", overwrite=False)
   ```
   
   It might be better that the default value for `overwrite` argument is `None` 
and update the mode if `overwrite is not None`?
   
   cc @HyukjinKwon 


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jerryshao commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API

2019-07-16 Thread GitBox
jerryshao commented on a change in pull request #25007: 
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API 
URL: https://github.com/apache/spark/pull/25007#discussion_r304230396
 
 

 ##
 File path: 
core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java
 ##
 @@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle.api;
+
+import org.apache.spark.annotation.Private;
+
+/**
+ * :: Experimental ::
+ * An interface for building shuffle support for Executors.
+ *
+ * @since 3.0.0
+ */
+@Private
+public interface ShuffleExecutorComponents {
+
+  /**
+   * Called once per executor to bootstrap this module with state that is 
specific to
+   * that executor, specifically the application ID and executor ID.
+   */
+  void initializeExecutor(String appId, String execId);
+
+  /**
+   * Returns the modules that are responsible for persisting shuffle data to 
the backing
+   * store.
+   * 
+   * This may be called multiple times on each executor. Implementations 
should not make
+   * any assumptions about the lifetime of the returned module.
+   */
+  ShuffleWriteSupport writes();
 
 Review comment:
   Is this better to use `writes` or `write`?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] gczsjdy commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API

2019-07-16 Thread GitBox
gczsjdy commented on a change in pull request #25007: 
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API 
URL: https://github.com/apache/spark/pull/25007#discussion_r304193877
 
 

 ##
 File path: 
core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java
 ##
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle.api;
+
+import java.io.IOException;
+
+import org.apache.spark.annotation.Private;
+
+/**
+ * :: Experimental ::
+ * A top-level writer that returns child writers for persisting the output of 
a map task,
+ * and then commits all of the writes as one atomic operation.
+ *
+ * @since 3.0.0
+ */
+@Private
+public interface ShuffleMapOutputWriter {
+
+  /**
+   * Creates a writer that can open an output stream to persist bytes for a 
given chunk of
+   * a map task.
+   * 
 
 Review comment:
   :nit remove ``s?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #24903: [SPARK-28084][SQL] Resolving the partition column name based on the resolver in sql load command

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #24903: [SPARK-28084][SQL] Resolving 
the partition column name based on the resolver in sql load command 
URL: https://github.com/apache/spark/pull/24903#issuecomment-512111605
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #24903: [SPARK-28084][SQL] Resolving the partition column name based on the resolver in sql load command

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #24903: [SPARK-28084][SQL] Resolving 
the partition column name based on the resolver in sql load command 
URL: https://github.com/apache/spark/pull/24903#issuecomment-512111609
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107769/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24903: [SPARK-28084][SQL] Resolving the partition column name based on the resolver in sql load command

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #24903: [SPARK-28084][SQL] Resolving the 
partition column name based on the resolver in sql load command 
URL: https://github.com/apache/spark/pull/24903#issuecomment-512111609
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107769/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24903: [SPARK-28084][SQL] Resolving the partition column name based on the resolver in sql load command

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #24903: [SPARK-28084][SQL] Resolving the 
partition column name based on the resolver in sql load command 
URL: https://github.com/apache/spark/pull/24903#issuecomment-512111605
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #24903: [SPARK-28084][SQL] Resolving the partition column name based on the resolver in sql load command

2019-07-16 Thread GitBox
SparkQA removed a comment on issue #24903: [SPARK-28084][SQL] Resolving the 
partition column name based on the resolver in sql load command 
URL: https://github.com/apache/spark/pull/24903#issuecomment-512077360
 
 
   **[Test build #107769 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107769/testReport)**
 for PR 24903 at commit 
[`261009e`](https://github.com/apache/spark/commit/261009e0035f17d6c0c42431d6c747e39d7913e9).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24903: [SPARK-28084][SQL] Resolving the partition column name based on the resolver in sql load command

2019-07-16 Thread GitBox
SparkQA commented on issue #24903: [SPARK-28084][SQL] Resolving the partition 
column name based on the resolver in sql load command 
URL: https://github.com/apache/spark/pull/24903#issuecomment-51279
 
 
   **[Test build #107769 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107769/testReport)**
 for PR 24903 at commit 
[`261009e`](https://github.com/apache/spark/commit/261009e0035f17d6c0c42431d6c747e39d7913e9).
* 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512107022
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107774/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512107022
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107774/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
SparkQA removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512101822
 
 
   **[Test build #107774 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107774/testReport)**
 for PR 25175 at commit 
[`8f3074f`](https://github.com/apache/spark/commit/8f3074ffee2553b9b62df5d9e1cf7b69cd93cde3).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512107019
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512107019
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster polling of executor memory metrics.

2019-07-16 Thread GitBox
wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster 
polling of executor memory metrics.
URL: https://github.com/apache/spark/pull/23767#discussion_r304223766
 
 

 ##
 File path: core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
 ##
 @@ -600,8 +611,16 @@ private[spark] object JsonProtocolSuite extends 
Assertions {
 assert(stageAttemptId1 === stageAttemptId2)
 assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => 
a.equals(b))
   })
-assertOptionEquals(e1.executorUpdates, e2.executorUpdates,
-(e1: ExecutorMetrics, e2: ExecutorMetrics) => assertEquals(e1, e2))
+assertSeqEquals[((Int, Int), ExecutorMetrics)](
+  e1.executorUpdates.toSeq.sortWith((x, y) => lexOrder(x._1, y._1)),
+  e2.executorUpdates.toSeq.sortWith((x, y) => lexOrder(x._1, y._1)),
 
 Review comment:
   I see. Ok.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with 
overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512106810
 
 
   **[Test build #107774 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107774/testReport)**
 for PR 25175 at commit 
[`8f3074f`](https://github.com/apache/spark/commit/8f3074ffee2553b9b62df5d9e1cf7b69cd93cde3).
* 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster polling of executor memory metrics.

2019-07-16 Thread GitBox
wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster 
polling of executor memory metrics.
URL: https://github.com/apache/spark/pull/23767#discussion_r304222908
 
 

 ##
 File path: 
core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
 ##
 @@ -268,12 +278,16 @@ private[spark] class EventLoggingListener(
 
   override def onExecutorMetricsUpdate(event: 
SparkListenerExecutorMetricsUpdate): Unit = {
 if (shouldLogStageExecutorMetrics) {
-  // For the active stages, record any new peak values for the memory 
metrics for the executor
-  event.executorUpdates.foreach { executorUpdates =>
-liveStageExecutorMetrics.values.foreach { peakExecutorMetrics =>
-  val peakMetrics = peakExecutorMetrics.getOrElseUpdate(
-event.execId, new ExecutorMetrics())
-  peakMetrics.compareAndUpdatePeakValues(executorUpdates)
+  event.executorUpdates.foreach { case (stageKey1, peaks) =>
+liveStageExecutorMetrics.foreach { case (stageKey2, 
metricsPerExecutor) =>
+  // If the update came from the driver, stageKey1 will be the dummy 
key (-1, -1),
+  // so record those peaks for all active stages.
+  // Otherwise, record the peaks for the matching stage.
+  if (stageKey1 == DRIVER_STAGE_KEY || stageKey1 == stageKey2) {
+val metrics = metricsPerExecutor.getOrElseUpdate(
+  event.execId, new ExecutorMetrics())
+metrics.compareAndUpdatePeakValues(peaks)
 
 Review comment:
   Ok.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster polling of executor memory metrics.

2019-07-16 Thread GitBox
wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster 
polling of executor memory metrics.
URL: https://github.com/apache/spark/pull/23767#discussion_r304222403
 
 

 ##
 File path: 
core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala
 ##
 @@ -0,0 +1,196 @@
+/*
+ * 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.executor
+
+import java.lang.Long.{MAX_VALUE => LONG_MAX_VALUE}
+import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
+import java.util.concurrent.atomic.{AtomicLong, AtomicLongArray}
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.internal.Logging
+import org.apache.spark.memory.MemoryManager
+import org.apache.spark.metrics.ExecutorMetricType
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+/**
+ * :: DeveloperApi ::
+ * A class that polls executor metrics, and tracks their peaks per task and 
per stage.
+ * Each executor keeps an instance of this class.
+ * The poll method polls the executor metrics, and is either run in its own 
thread or
+ * called by the executor's heartbeater thread, depending on configuration.
+ * The class keeps two ConcurrentHashMaps that are accessed (via its methods) 
by the
+ * executor's task runner threads concurrently with the polling thread. One 
thread may
+ * update one of these maps while another reads it, so the reading thread may 
not get
+ * the latest metrics, but this is ok.
+ *
+ * @param memoryManager the memory manager used by the executor.
+ * @param pollingInterval the polling interval in milliseconds.
+ */
+@DeveloperApi
+private[spark] class ExecutorMetricsPoller(
+memoryManager: MemoryManager,
+pollingInterval: Long)
+  extends Logging {
+
+  type StageKey = (Int, Int)
+  // tuple for Task Count and Metric Peaks
+  type TCMP = (AtomicLong, AtomicLongArray)
+
+  // Map of (stageId, stageAttemptId) to (count of running tasks, executor 
metric peaks)
+  private val stageTCMP = new ConcurrentHashMap[StageKey, TCMP]
+
+  // Map of taskId to executor metric peaks
+  private val taskMetricPeaks = new ConcurrentHashMap[Long, AtomicLongArray]
+
+  private val poller =
+if (pollingInterval > 0) {
+  
Some(ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-metrics-poller"))
+} else {
+  None
+}
+
+  /**
+   * Function to poll executor metrics.
+   * On start, if pollingInterval is positive, this is scheduled to run at 
that interval.
+   * Otherwise, this is called by the reportHeartBeat function defined in 
Executor and passed
+   * to its Heartbeater.
+   */
+  def poll(): Unit = {
+// Note: Task runner threads may update stageTCMP or read from 
taskMetricPeaks concurrently
+// with this function via calls to methods of this class.
+
+// get the latest values for the metrics
+val latestMetrics = ExecutorMetrics.getCurrentMetrics(memoryManager)
+
+def updatePeaks(metrics: AtomicLongArray): Unit = {
+  (0 until metrics.length).foreach { i =>
+metrics.getAndAccumulate(i, latestMetrics(i), math.max)
+  }
+}
+
+// for each active stage, update the peaks
+stageTCMP.forEachValue(LONG_MAX_VALUE, v => updatePeaks(v._2))
+
+// for each running task, update the peaks
+taskMetricPeaks.forEachValue(LONG_MAX_VALUE, updatePeaks)
+  }
+
+  /** Starts the polling thread. */
+  def start(): Unit = {
+poller.foreach { exec =>
+  val pollingTask: Runnable = () => Utils.logUncaughtExceptions(poll())
+  exec.scheduleAtFixedRate(pollingTask, 0L, pollingInterval, 
TimeUnit.MILLISECONDS)
+}
+  }
+
+  /**
+   * Called by TaskRunner#run.
+   *
+   * @param taskId the id of the task being run.
+   * @param stageId the id of the stage the task belongs to.
+   * @param stageAttemptId the attempt number of the stage the task belongs to.
+   */
+  def onTaskStart(taskId: Long, stageId: Int, stageAttemptId: Int): Unit = {
+// Put an entry in taskMetricPeaks for the task.
+taskMetricPeaks.put(taskId, new 
AtomicLongArray(ExecutorMetricType.numMetrics))
+
+// Put a new entry in stageTCMP for the stage if there isn't one already.
+// Increment the task cou

[GitHub] [spark] wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster polling of executor memory metrics.

2019-07-16 Thread GitBox
wypoon commented on a change in pull request #23767: [SPARK-26329][CORE] Faster 
polling of executor memory metrics.
URL: https://github.com/apache/spark/pull/23767#discussion_r304222033
 
 

 ##
 File path: 
core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala
 ##
 @@ -0,0 +1,196 @@
+/*
+ * 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.executor
+
+import java.lang.Long.{MAX_VALUE => LONG_MAX_VALUE}
+import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
+import java.util.concurrent.atomic.{AtomicLong, AtomicLongArray}
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.internal.Logging
+import org.apache.spark.memory.MemoryManager
+import org.apache.spark.metrics.ExecutorMetricType
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+/**
+ * :: DeveloperApi ::
 
 Review comment:
   You're right. This is just internal implementation detail.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] yeshengm commented on a change in pull request #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
yeshengm commented on a change in pull request #25164: [SPARK-28375][SQL] 
Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates 
if run multiple times
URL: https://github.com/apache/spark/pull/25164#discussion_r304219902
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 ##
 @@ -275,13 +275,16 @@ object PullupCorrelatedPredicates extends 
Rule[LogicalPlan] with PredicateHelper
 plan transformExpressions {
   case ScalarSubquery(sub, children, exprId) if children.nonEmpty =>
 val (newPlan, newCond) = pullOutCorrelatedPredicates(sub, outerPlans)
-ScalarSubquery(newPlan, newCond, exprId)
+val conds = newCond ++ children.filter(_.isInstanceOf[Predicate])
 
 Review comment:
   Ahh, got it. Thanks for your detailed explanation!


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with 
overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512101822
 
 
   **[Test build #107774 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107774/testReport)**
 for PR 25175 at commit 
[`8f3074f`](https://github.com/apache/spark/commit/8f3074ffee2553b9b62df5d9e1cf7b69cd93cde3).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512101460
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12893/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] Udbhav30 commented on issue #24601: [SPARK-27702][K8S] Allow using some alternatives for service accounts

2019-07-16 Thread GitBox
Udbhav30 commented on issue #24601: [SPARK-27702][K8S] Allow using some 
alternatives for service accounts
URL: https://github.com/apache/spark/pull/24601#issuecomment-512101459
 
 
   Hii @dongjoon-hyun can you please check if we can merge 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512101460
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12893/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512101452
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512101452
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dilipbiswal commented on a change in pull request #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
dilipbiswal commented on a change in pull request #25164: [SPARK-28375][SQL] 
Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates 
if run multiple times
URL: https://github.com/apache/spark/pull/25164#discussion_r304217306
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 ##
 @@ -275,13 +275,16 @@ object PullupCorrelatedPredicates extends 
Rule[LogicalPlan] with PredicateHelper
 plan transformExpressions {
   case ScalarSubquery(sub, children, exprId) if children.nonEmpty =>
 val (newPlan, newCond) = pullOutCorrelatedPredicates(sub, outerPlans)
-ScalarSubquery(newPlan, newCond, exprId)
+val conds = newCond ++ children.filter(_.isInstanceOf[Predicate])
 
 Review comment:
   @yeshengm 
   > According to jira, it seems that spark-19712 is already resolved
   
   Yeah :-). As you can see, there are multiple PRs against the JIRA. It got 
marked resolved after the first PR, i think. But we are doing this work in 
stages. I am working on couple of more prs to complete this work.
   
   > That said, we can safely combine the two now, right?
   
   To the best of my knowledge, we can't combine the two until we move rewrite 
to the beginning of optimization process (this is the piece i am working on at 
the moment).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304215899
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BooleanExpressionsSuite.scala
 ##
 @@ -0,0 +1,54 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.BooleanTest._
+import org.apache.spark.sql.types._
+
+class BooleanExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
 
 Review comment:
   OK


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25164: [SPARK-28375][SQL] Prevent the 
PullupCorrelatedPredicates optimizer rule from removing predicates if run 
multiple times
URL: https://github.com/apache/spark/pull/25164#issuecomment-512097488
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107767/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25164: [SPARK-28375][SQL] Prevent the 
PullupCorrelatedPredicates optimizer rule from removing predicates if run 
multiple times
URL: https://github.com/apache/spark/pull/25164#issuecomment-512097484
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] yeshengm commented on a change in pull request #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
yeshengm commented on a change in pull request #25164: [SPARK-28375][SQL] 
Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates 
if run multiple times
URL: https://github.com/apache/spark/pull/25164#discussion_r304215574
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 ##
 @@ -275,13 +275,16 @@ object PullupCorrelatedPredicates extends 
Rule[LogicalPlan] with PredicateHelper
 plan transformExpressions {
   case ScalarSubquery(sub, children, exprId) if children.nonEmpty =>
 val (newPlan, newCond) = pullOutCorrelatedPredicates(sub, outerPlans)
-ScalarSubquery(newPlan, newCond, exprId)
+val conds = newCond ++ children.filter(_.isInstanceOf[Predicate])
 
 Review comment:
   @dilipbiswal According to jira, it seems that spark-19712 is already 
resolved. That said, we can safely combine the two now, right?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25164: [SPARK-28375][SQL] Prevent the 
PullupCorrelatedPredicates optimizer rule from removing predicates if run 
multiple times
URL: https://github.com/apache/spark/pull/25164#issuecomment-512097484
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25160: [SPARK-28399][ML] implement RobustScaler

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25160: [SPARK-28399][ML] implement 
RobustScaler
URL: https://github.com/apache/spark/pull/25160#issuecomment-512097471
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107770/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25160: [SPARK-28399][ML] implement RobustScaler

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25160: [SPARK-28399][ML] implement 
RobustScaler
URL: https://github.com/apache/spark/pull/25160#issuecomment-512097465
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25160: [SPARK-28399][ML] implement RobustScaler

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25160: [SPARK-28399][ML] implement 
RobustScaler
URL: https://github.com/apache/spark/pull/25160#issuecomment-512097471
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107770/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25160: [SPARK-28399][ML] implement RobustScaler

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25160: [SPARK-28399][ML] implement 
RobustScaler
URL: https://github.com/apache/spark/pull/25160#issuecomment-512097465
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25164: [SPARK-28375][SQL] Prevent the 
PullupCorrelatedPredicates optimizer rule from removing predicates if run 
multiple times
URL: https://github.com/apache/spark/pull/25164#issuecomment-512097488
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107767/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25160: [SPARK-28399][ML] implement RobustScaler

2019-07-16 Thread GitBox
SparkQA commented on issue #25160: [SPARK-28399][ML] implement RobustScaler
URL: https://github.com/apache/spark/pull/25160#issuecomment-512097276
 
 
   **[Test build #107770 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107770/testReport)**
 for PR 25160 at commit 
[`bcc10cd`](https://github.com/apache/spark/commit/bcc10cd2d4ffe8c3922996dc077b0c90d64150e7).
* 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
SparkQA removed a comment on issue #25164: [SPARK-28375][SQL] Prevent the 
PullupCorrelatedPredicates optimizer rule from removing predicates if run 
multiple times
URL: https://github.com/apache/spark/pull/25164#issuecomment-512061818
 
 
   **[Test build #107767 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107767/testReport)**
 for PR 25164 at commit 
[`4d405f6`](https://github.com/apache/spark/commit/4d405f62b843f84aa2ba2049df2b6a3f9b01f476).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25160: [SPARK-28399][ML] implement RobustScaler

2019-07-16 Thread GitBox
SparkQA removed a comment on issue #25160: [SPARK-28399][ML] implement 
RobustScaler
URL: https://github.com/apache/spark/pull/25160#issuecomment-512086889
 
 
   **[Test build #107770 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107770/testReport)**
 for PR 25160 at commit 
[`bcc10cd`](https://github.com/apache/spark/commit/bcc10cd2d4ffe8c3922996dc077b0c90d64150e7).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512096985
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107771/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512096981
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
SparkQA commented on issue #25164: [SPARK-28375][SQL] Prevent the 
PullupCorrelatedPredicates optimizer rule from removing predicates if run 
multiple times
URL: https://github.com/apache/spark/pull/25164#issuecomment-512097138
 
 
   **[Test build #107767 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107767/testReport)**
 for PR 25164 at commit 
[`4d405f6`](https://github.com/apache/spark/commit/4d405f62b843f84aa2ba2049df2b6a3f9b01f476).
* 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512096985
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107771/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304214922
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/booleanExpressions.scala
 ##
 @@ -0,0 +1,82 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * String to indicate which boolean test selected.
+ */
+object BooleanTest {
+  val TRUE = "TRUE"
+  val FALSE = "FALSE"
+  val UNKNOWN = "UNKNOWN"
+
+  def calculate(input: Any, booleanValue: String): Boolean = {
+booleanValue match {
+  case TRUE => input == true
+  case FALSE => input == false
+  case UNKNOWN => input == null
+  case _ => throw new AnalysisException("Boolean test value must be one of 
TRUE, " +
+"FALSE and UNKNOWN.")
+}
+  }
+}
+
+/**
+ * Test the value of an expression is true, false, or unknown.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(expr, booleanValue) - Returns true if `expr` equals 
booleanValue, " +
+"or false otherwise.",
+  arguments = """
+Arguments:
+  * expr - a boolean expression
+  * booleanValue - a boolean value represented by a string. booleanValue 
must be one
+  of TRUE, FALSE and UNKNOWN.
+  """,
+  examples = """
+Examples:
+> SELECT _FUNC_(1 > 2, true);
+   false
+> SELECT _FUNC_(2 > 1, true);
+   true
+  """)
 
 Review comment:
   OK.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304214964
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 ##
 @@ -1223,6 +1223,12 @@ class AstBuilder(conf: SQLConf) extends 
SqlBaseBaseVisitor[AnyRef] with Logging
   case c: CreateNamedStruct => c.valExprs
   case other => Seq(other)
 }
+// Check the argument of boolean test is valid.
+def checkBooleanTestArgs(e: Expression): Unit = e.dataType match {
 
 Review comment:
   OK


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
SparkQA removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512092300
 
 
   **[Test build #107771 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107771/testReport)**
 for PR 25175 at commit 
[`379eba6`](https://github.com/apache/spark/commit/379eba661c4b7734bd216a65c5fe162224ba8e64).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512096981
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with 
overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512096806
 
 
   **[Test build #107771 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107771/testReport)**
 for PR 25175 at commit 
[`379eba6`](https://github.com/apache/spark/commit/379eba661c4b7734bd216a65c5fe162224ba8e64).
* 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304214658
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/booleanExpressions.scala
 ##
 @@ -0,0 +1,82 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * String to indicate which boolean test selected.
+ */
+object BooleanTest {
+  val TRUE = "TRUE"
+  val FALSE = "FALSE"
+  val UNKNOWN = "UNKNOWN"
+
+  def calculate(input: Any, booleanValue: String): Boolean = {
+booleanValue match {
+  case TRUE => input == true
+  case FALSE => input == false
+  case UNKNOWN => input == null
+  case _ => throw new AnalysisException("Boolean test value must be one of 
TRUE, " +
+"FALSE and UNKNOWN.")
+}
+  }
+}
 
 Review comment:
   @dongjoon-hyun Thanks for your example. I get 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dilipbiswal commented on a change in pull request #25164: [SPARK-28375][SQL] Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates if run multiple times

2019-07-16 Thread GitBox
dilipbiswal commented on a change in pull request #25164: [SPARK-28375][SQL] 
Prevent the PullupCorrelatedPredicates optimizer rule from removing predicates 
if run multiple times
URL: https://github.com/apache/spark/pull/25164#discussion_r304214534
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 ##
 @@ -275,13 +275,16 @@ object PullupCorrelatedPredicates extends 
Rule[LogicalPlan] with PredicateHelper
 plan transformExpressions {
   case ScalarSubquery(sub, children, exprId) if children.nonEmpty =>
 val (newPlan, newCond) = pullOutCorrelatedPredicates(sub, outerPlans)
-ScalarSubquery(newPlan, newCond, exprId)
+val conds = newCond ++ children.filter(_.isInstanceOf[Predicate])
 
 Review comment:
   @yeshengm So today, rewritePredicateSubquery happens almost at the end of 
optimization process where as PullupCorrelatedPredicates happens at the 
beginning of optimization process. In SPARK-19712, we are attempting to move 
rewritePredicateSubquery to the beginning of optimizer. Once that happens, i 
don't anticipate any problem in combining the two.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] Support ANSI SQL Boolean-Predicate syntax

2019-07-16 Thread GitBox
beliefer commented on a change in pull request #25074: [SPARK-27924][SQL] 
Support ANSI SQL Boolean-Predicate syntax 
URL: https://github.com/apache/spark/pull/25074#discussion_r304213725
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/booleanExpressions.scala
 ##
 @@ -0,0 +1,82 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
ExprCode}
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * String to indicate which boolean test selected.
+ */
+object BooleanTest {
+  val TRUE = "TRUE"
+  val FALSE = "FALSE"
+  val UNKNOWN = "UNKNOWN"
+
+  def calculate(input: Any, booleanValue: String): Boolean = {
+booleanValue match {
+  case TRUE => input == true
+  case FALSE => input == false
+  case UNKNOWN => input == null
+  case _ => throw new AnalysisException("Boolean test value must be one of 
TRUE, " +
+"FALSE and UNKNOWN.")
+}
+  }
+}
+
+/**
+ * Test the value of an expression is true, false, or unknown.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(expr, booleanValue) - Returns true if `expr` equals 
booleanValue, " +
+"or false otherwise.",
+  arguments = """
+Arguments:
+  * expr - a boolean expression
+  * booleanValue - a boolean value represented by a string. booleanValue 
must be one
+  of TRUE, FALSE and UNKNOWN.
+  """,
+  examples = """
+Examples:
+> SELECT _FUNC_(1 > 2, true);
+   false
+> SELECT _FUNC_(2 > 1, true);
+   true
+  """)
+case class BooleanTest(child: Expression, booleanValue: String)
 
 Review comment:
   OK. I will change 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making UDFs (virtually) no-op

2019-07-16 Thread GitBox
SparkQA commented on issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make 
integrated UDF tests robust by making UDFs (virtually) no-op
URL: https://github.com/apache/spark/pull/25130#issuecomment-512094840
 
 
   **[Test build #107773 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107773/testReport)**
 for PR 25130 at commit 
[`720eea7`](https://github.com/apache/spark/commit/720eea7bbd95c74065add08f6195f78d355d7e3a).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array

2019-07-16 Thread GitBox
SparkQA commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY 
function support byte array
URL: https://github.com/apache/spark/pull/25172#issuecomment-512094841
 
 
   **[Test build #107772 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107772/testReport)**
 for PR 25172 at commit 
[`702f131`](https://github.com/apache/spark/commit/702f13161d8fb1ce5cc8a585d11e2887f567c49f).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making UDFs (virtually) no-op

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25130: 
[SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making 
UDFs (virtually) no-op
URL: https://github.com/apache/spark/pull/25130#issuecomment-512094487
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25172: [SPARK-28412][SQL] ANSI SQL: 
OVERLAY function support byte array
URL: https://github.com/apache/spark/pull/25172#issuecomment-512094517
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making UDFs (virtually) no-op

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25130: 
[SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making 
UDFs (virtually) no-op
URL: https://github.com/apache/spark/pull/25130#issuecomment-512094490
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12892/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25172: [SPARK-28412][SQL] ANSI SQL: 
OVERLAY function support byte array
URL: https://github.com/apache/spark/pull/25172#issuecomment-512094521
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12891/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY 
function support byte array
URL: https://github.com/apache/spark/pull/25172#issuecomment-512094521
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12891/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making UDFs (virtually) no-op

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make 
integrated UDF tests robust by making UDFs (virtually) no-op
URL: https://github.com/apache/spark/pull/25130#issuecomment-512094487
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make integrated UDF tests robust by making UDFs (virtually) no-op

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25130: [SPARK-28359][SQL][PYTHON][TESTS] Make 
integrated UDF tests robust by making UDFs (virtually) no-op
URL: https://github.com/apache/spark/pull/25130#issuecomment-512094490
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12892/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY 
function support byte array
URL: https://github.com/apache/spark/pull/25172#issuecomment-512094517
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maropu commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array

2019-07-16 Thread GitBox
maropu commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function 
support byte array
URL: https://github.com/apache/spark/pull/25172#issuecomment-512094138
 
 
   ok, thanks! I'll review later.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
HyukjinKwon commented on a change in pull request #25175: 
[SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#discussion_r304212498
 
 

 ##
 File path: python/pyspark/sql/tests/test_readwriter.py
 ##
 @@ -141,6 +141,26 @@ def count_bucketed_cols(names, table="pyspark_bucket"):
 .mode("overwrite").saveAsTable("pyspark_bucket"))
 self.assertSetEqual(set(data), 
set(self.spark.table("pyspark_bucket").collect()))
 
+def test_insert_into(self):
+df = self.spark.createDataFrame([("a", 1), ("b", 2)], ["C1", "C2"])
+with self.table("test_table"):
+df.write.saveAsTable("test_table")
+self.assertEqual(2, self.spark.sql("select * from 
test_table").count())
+
+df.write.insertInto("test_table")
+self.assertEqual(4, self.spark.sql("select * from 
test_table").count())
+
+df.write.mode("overwrite").insertInto("test_table")
+self.assertEqual(2, self.spark.sql("select * from 
test_table").count())
+
+df.write.insertInto("test_table", True)
+self.assertEqual(2, self.spark.sql("select * from 
test_table").count())
+
+df.write.insertInto("test_table", False)
+self.assertEqual(4, self.spark.sql("select * from 
test_table").count())
+
+# self.spark.sql("drop table test_table")
 
 Review comment:
   seems a mistake.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512093246
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12890/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25175: [SPARK-28411][PYTHON][SQL] 
InsertInto with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512093242
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512093242
 
 
   Merged build finished. Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
AmplabJenkins commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto 
with overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512093246
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/12890/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jerryshao commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API

2019-07-16 Thread GitBox
jerryshao commented on a change in pull request #25007: 
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API 
URL: https://github.com/apache/spark/pull/25007#discussion_r304211829
 
 

 ##
 File path: 
core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java
 ##
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle.api;
+
+import java.io.IOException;
+
+import org.apache.spark.annotation.Private;
+
+/**
+ * :: Experimental ::
 
 Review comment:
   I think here and other places should also change to `:: Private ::`


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jerryshao commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API

2019-07-16 Thread GitBox
jerryshao commented on a change in pull request #25007: 
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API 
URL: https://github.com/apache/spark/pull/25007#discussion_r304211829
 
 

 ##
 File path: 
core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java
 ##
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle.api;
+
+import java.io.IOException;
+
+import org.apache.spark.annotation.Private;
+
+/**
+ * :: Experimental ::
 
 Review comment:
   I think here should also change to `:: Private ::`


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with overwrite is not honored

2019-07-16 Thread GitBox
SparkQA commented on issue #25175: [SPARK-28411][PYTHON][SQL] InsertInto with 
overwrite is not honored
URL: https://github.com/apache/spark/pull/25175#issuecomment-512092300
 
 
   **[Test build #107771 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107771/testReport)**
 for PR 25175 at commit 
[`379eba6`](https://github.com/apache/spark/commit/379eba661c4b7734bd216a65c5fe162224ba8e64).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] williammoy commented on issue #23460: [SPARK-26544][SQL] Escape string to keep alignment with hive

2019-07-16 Thread GitBox
williammoy commented on issue #23460: [SPARK-26544][SQL] Escape string to keep 
alignment with hive
URL: https://github.com/apache/spark/pull/23460#issuecomment-512092298
 
 
   Hi guys, is there any update on 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array

2019-07-16 Thread GitBox
beliefer commented on issue #25172: [SPARK-28412][SQL] ANSI SQL: OVERLAY 
function support byte array
URL: https://github.com/apache/spark/pull/25172#issuecomment-512091872
 
 
   > Is this related to the standard?
   
   Yes. ANSI SQL T312
   ```
::=
   OVERLAY   PLACING 
   FROM  [ FOR  ] 
   ```


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25134: [SPARK-28366][CORE] Logging in driver when loading single large unsplittable file

2019-07-16 Thread GitBox
AmplabJenkins removed a comment on issue #25134: [SPARK-28366][CORE] Logging in 
driver when loading single large unsplittable file
URL: https://github.com/apache/spark/pull/25134#issuecomment-512091386
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/107768/
   Test PASSed.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
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 issue #25134: [SPARK-28366][CORE] Logging in driver when loading single large unsplittable file

2019-07-16 Thread GitBox
SparkQA removed a comment on issue #25134: [SPARK-28366][CORE] Logging in 
driver when loading single large unsplittable file
URL: https://github.com/apache/spark/pull/25134#issuecomment-512066779
 
 
   **[Test build #107768 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/107768/testReport)**
 for PR 25134 at commit 
[`736587b`](https://github.com/apache/spark/commit/736587beada1d3796f376d41bfec625b3d3837af).


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



  1   2   3   4   5   6   7   8   9   >