[GitHub] [spark] s1ck commented on a change in pull request #24851: [SPARK-27303][GRAPH] Add PropertyGraph construction API

2019-06-27 Thread GitBox
s1ck commented on a change in pull request #24851: [SPARK-27303][GRAPH] Add 
PropertyGraph construction API
URL: https://github.com/apache/spark/pull/24851#discussion_r298473453
 
 

 ##
 File path: 
graph/api/src/main/scala/org/apache/spark/graph/api/CypherSession.scala
 ##
 @@ -0,0 +1,138 @@
+/*
+ * 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.graph.api
+
+import scala.collection.JavaConverters
+
+import org.apache.spark.sql.{functions, DataFrame, SparkSession}
+
+object CypherSession {
+  val ID_COLUMN = "$ID"
+  val SOURCE_ID_COLUMN = "$SOURCE_ID"
+  val TARGET_ID_COLUMN = "$TARGET_ID"
+  val LABEL_COLUMN_PREFIX = ":"
+}
+
+/**
+ * The entry point for using property graphs in Spark.
+ *
+ * Provides factory methods for creating [[PropertyGraph]] instances.
+ *
+ * Wraps a [[org.apache.spark.sql.SparkSession]].
+ *
+ * @since 3.0.0
+ */
+trait CypherSession {
+
+  def sparkSession: SparkSession
+
+  /**
+   * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and 
[[RelationshipFrame]]s.
+   * At least one [[NodeFrame]] has to be provided.
+   *
+   * For each label set and relationship type there can be at most one 
[[NodeFrame]] and at most one
+   * [[RelationshipFrame]], respectively.
+   *
+   * @param nodes NodeFrames that define the nodes in the graph
+   * @param relationships RelationshipFrames that define the relationships in 
the graph
+   * @since 3.0.0
+   */
+  def createGraph(nodes: Seq[NodeFrame], relationships: 
Seq[RelationshipFrame]): PropertyGraph
+
+  /**
+   * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and 
[[RelationshipFrame]]s.
+   * At least one [[NodeFrame]] has to be provided.
+   *
+   * For each label set and relationship type there can be at most one 
[[NodeFrame]] and at most one
+   * [[RelationshipFrame]], respectively.
+   *
+   * @param nodes NodeFrames that define the nodes in the graph
+   * @param relationships RelationshipFrames that define the relationships in 
the graph
+   * @since 3.0.0
+   */
+  def createGraph(
+  nodes: java.util.List[NodeFrame],
+  relationships: java.util.List[RelationshipFrame]): PropertyGraph = {
+createGraph(JavaConverters.asScalaBuffer(nodes), 
JavaConverters.asScalaBuffer(relationships))
+  }
+
+  /**
+   * Creates a [[PropertyGraph]] from nodes and relationships.
+   *
+   * The given DataFrames need to adhere to the following column naming 
conventions:
+   *
+   * {{{
+   * Id column:`$ID`(nodes and relationships)
+   * SourceId column:  `$SOURCE_ID` (relationships)
+   * TargetId column:  `$TARGET_ID` (relationships)
+   *
+   * Label columns:`:{LABEL_NAME}`  (nodes)
+   * RelType columns:  `:{REL_TYPE}`(relationships)
+   *
+   * Property columns: `{Property_Key}` (nodes and relationships)
+   * }}}
+   *
+   * @see [[CypherSession]]
+   * @param nodes node DataFrame
+   * @param relationships relationship DataFrame
+   * @since 3.0.0
+   */
+  def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = 
{
+val idColumn = CypherSession.ID_COLUMN
+val sourceIdColumn = CypherSession.SOURCE_ID_COLUMN
+val targetIdColumn = CypherSession.TARGET_ID_COLUMN
+
+val labelColumns = 
nodes.columns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)).toSet
+val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns)
+  .map(col => col -> col)
+  .toMap
+
+val trueLit = functions.lit(true)
+val falseLit = functions.lit(false)
+
+val labelSets = labelColumns.subsets().toSet + Set.empty
+val nodeFrames = labelSets.map { labelSet =>
+  val predicate = labelColumns
+.map {
+  case labelColumn if labelSet.contains(labelColumn) => 
nodes.col(labelColumn) === trueLit
+  case labelColumn => nodes.col(labelColumn) === falseLit
+}
+.reduce(_ && _)
+
+  NodeFrame(nodes.filter(predicate), idColumn, 
labelSet.map(_.substring(1)), nodeProperties)
+}
+
+val relColumns = relationships.columns.toSet
+val relTypeColumns = 
relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX))
+val propertyColumns

[GitHub] [spark] s1ck commented on a change in pull request #24851: [SPARK-27303][GRAPH] Add PropertyGraph construction API

2019-06-27 Thread GitBox
s1ck commented on a change in pull request #24851: [SPARK-27303][GRAPH] Add 
PropertyGraph construction API
URL: https://github.com/apache/spark/pull/24851#discussion_r298472937
 
 

 ##
 File path: 
graph/api/src/main/scala/org/apache/spark/graph/api/CypherSession.scala
 ##
 @@ -0,0 +1,138 @@
+/*
+ * 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.graph.api
+
+import scala.collection.JavaConverters
+
+import org.apache.spark.sql.{functions, DataFrame, SparkSession}
+
+object CypherSession {
+  val ID_COLUMN = "$ID"
+  val SOURCE_ID_COLUMN = "$SOURCE_ID"
+  val TARGET_ID_COLUMN = "$TARGET_ID"
+  val LABEL_COLUMN_PREFIX = ":"
+}
+
+/**
+ * The entry point for using property graphs in Spark.
+ *
+ * Provides factory methods for creating [[PropertyGraph]] instances.
+ *
+ * Wraps a [[org.apache.spark.sql.SparkSession]].
+ *
+ * @since 3.0.0
+ */
+trait CypherSession {
+
+  def sparkSession: SparkSession
+
+  /**
+   * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and 
[[RelationshipFrame]]s.
+   * At least one [[NodeFrame]] has to be provided.
+   *
+   * For each label set and relationship type there can be at most one 
[[NodeFrame]] and at most one
+   * [[RelationshipFrame]], respectively.
+   *
+   * @param nodes NodeFrames that define the nodes in the graph
+   * @param relationships RelationshipFrames that define the relationships in 
the graph
+   * @since 3.0.0
+   */
+  def createGraph(nodes: Seq[NodeFrame], relationships: 
Seq[RelationshipFrame]): PropertyGraph
+
+  /**
+   * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and 
[[RelationshipFrame]]s.
+   * At least one [[NodeFrame]] has to be provided.
+   *
+   * For each label set and relationship type there can be at most one 
[[NodeFrame]] and at most one
+   * [[RelationshipFrame]], respectively.
+   *
+   * @param nodes NodeFrames that define the nodes in the graph
+   * @param relationships RelationshipFrames that define the relationships in 
the graph
+   * @since 3.0.0
+   */
+  def createGraph(
+  nodes: java.util.List[NodeFrame],
+  relationships: java.util.List[RelationshipFrame]): PropertyGraph = {
+createGraph(JavaConverters.asScalaBuffer(nodes), 
JavaConverters.asScalaBuffer(relationships))
+  }
+
+  /**
+   * Creates a [[PropertyGraph]] from nodes and relationships.
+   *
+   * The given DataFrames need to adhere to the following column naming 
conventions:
+   *
+   * {{{
+   * Id column:`$ID`(nodes and relationships)
+   * SourceId column:  `$SOURCE_ID` (relationships)
+   * TargetId column:  `$TARGET_ID` (relationships)
+   *
+   * Label columns:`:{LABEL_NAME}`  (nodes)
+   * RelType columns:  `:{REL_TYPE}`(relationships)
+   *
+   * Property columns: `{Property_Key}` (nodes and relationships)
+   * }}}
+   *
+   * @see [[CypherSession]]
+   * @param nodes node DataFrame
+   * @param relationships relationship DataFrame
+   * @since 3.0.0
+   */
+  def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = 
{
+val idColumn = CypherSession.ID_COLUMN
+val sourceIdColumn = CypherSession.SOURCE_ID_COLUMN
+val targetIdColumn = CypherSession.TARGET_ID_COLUMN
+
+val labelColumns = 
nodes.columns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)).toSet
+val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns)
+  .map(col => col -> col)
+  .toMap
+
+val trueLit = functions.lit(true)
+val falseLit = functions.lit(false)
+
+val labelSets = labelColumns.subsets().toSet + Set.empty
 
 Review comment:
   Each of those sets represents a possible label combination (the empty set is 
the case where a node has no label). The input dataframe has no information 
about the possible label combinations. From looking at its schema, you can just 
derive which labels exist and we have to assume that any combination of those 
labels exists in the data. The alternative would be to look at the data, which 
we want to avoid.
   
   Generally, this method is just for convenience. To avoid the combinatorical 
problem, one can use `NodeFrame` and `RelationshipFrame` for each

[GitHub] [spark] cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] Migrate Kafka Batch source to v2.

2019-06-27 Thread GitBox
cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] 
Migrate Kafka Batch source to v2.
URL: https://github.com/apache/spark/pull/24738#discussion_r298470767
 
 

 ##
 File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala
 ##
 @@ -299,7 +299,7 @@ private[kafka010] class KafkaMicroBatchStream(
   if (content(0) == 'v') {
 val indexOfNewLine = content.indexOf("\n")
 if (indexOfNewLine > 0) {
-  val version = parseVersion(content.substring(0, indexOfNewLine), 
VERSION)
+  parseVersion(content.substring(0, indexOfNewLine), VERSION)
 
 Review comment:
   so it's not parse but validate?


This is an automated message from the 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] cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] Migrate Kafka Batch source to v2.

2019-06-27 Thread GitBox
cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] 
Migrate Kafka Batch source to v2.
URL: https://github.com/apache/spark/pull/24738#discussion_r298467027
 
 

 ##
 File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala
 ##
 @@ -0,0 +1,103 @@
+/*
+ * 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.kafka010
+
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT
+import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, 
PartitionReaderFactory}
+
+
+private[kafka010] class KafkaBatch(
+strategy: ConsumerStrategy,
+sourceOptions: Map[String, String],
+specifiedKafkaParams: Map[String, String],
+failOnDataLoss: Boolean,
+startingOffsets: KafkaOffsetRangeLimit,
+endingOffsets: KafkaOffsetRangeLimit)
+extends Batch with Logging {
 
 Review comment:
   nit: 2 space indentation for `extends` 


This is an automated message from the 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] cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] Migrate Kafka Batch source to v2.

2019-06-27 Thread GitBox
cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] 
Migrate Kafka Batch source to v2.
URL: https://github.com/apache/spark/pull/24738#discussion_r298468819
 
 

 ##
 File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
 ##
 @@ -353,15 +354,18 @@ private[kafka010] class KafkaSourceProvider extends 
DataSourceRegister
 }
   }
 
-  class KafkaTable(strategy: => ConsumerStrategy) extends Table
-with SupportsRead with SupportsWrite {
+  class KafkaTable extends Table with SupportsRead with SupportsWrite {
 
-override def name(): String = s"Kafka $strategy"
+override def name(): String = "KafkaTable"
 
 Review comment:
   should we at least put topic name in the kafka table name?


This is an automated message from the 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] cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] Migrate Kafka Batch source to v2.

2019-06-27 Thread GitBox
cloud-fan commented on a change in pull request #24738: [SPARK-23098][SQL] 
Migrate Kafka Batch source to v2.
URL: https://github.com/apache/spark/pull/24738#discussion_r298470645
 
 

 ##
 File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
 ##
 @@ -212,8 +213,14 @@ class DataFrameReader private[sql](sparkSession: 
SparkSession) extends Logging {
 f.fallbackFileFormat
   case _ => lookupCls
 }
+val clsMustUseV1Source = cls.newInstance() match {
+  case f: DataSourceRegister if useV1Sources.contains(f.shortName()) ||
 
 Review comment:
   cc @gengliangwang  the `useV1SourceReaderList` should be applied to all data 
sources not only file sources, according to the doc. Can you fix it? The 
current patch works but it's not a clean solution.


This is an automated message from the 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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
beliefer commented on issue #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY 
function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506613258
 
 
   @ueshin Thanks for your review. I learned more.


This is an automated message from the 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 edited a comment on issue #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
beliefer edited a comment on issue #24918: [SPARK-28077][SQL] Support ANSI SQL 
OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506613258
 
 
   @ueshin Thanks for your review. I learned more. I have resolved the last two 
nits.


This is an automated message from the 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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24986: [SPARK-28185][ML][SQL] Trigger 
pandas iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506611181
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106986/
   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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
SparkQA removed a comment on issue #24986: [SPARK-28185][ML][SQL] Trigger 
pandas iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506604697
 
 
   **[Test build #106986 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106986/testReport)**
 for PR 24986 at commit 
[`d2aa561`](https://github.com/apache/spark/commit/d2aa561130d994b6cde4f915af547c6fb9330cc9).


This is an automated message from the 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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24986: [SPARK-28185][ML][SQL] Trigger 
pandas iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506611177
 
 
   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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24986: [SPARK-28185][ML][SQL] Trigger pandas 
iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506611181
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106986/
   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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24986: [SPARK-28185][ML][SQL] Trigger pandas 
iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506611177
 
 
   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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
SparkQA commented on issue #24986: [SPARK-28185][ML][SQL] Trigger pandas 
iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506610924
 
 
   **[Test build #106986 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106986/testReport)**
 for PR 24986 at commit 
[`d2aa561`](https://github.com/apache/spark/commit/d2aa561130d994b6cde4f915af547c6fb9330cc9).
* 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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24918: [SPARK-28077][SQL] Support 
ANSI SQL OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506610229
 
 
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24918: [SPARK-28077][SQL] Support 
ANSI SQL OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506610233
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106984/
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24918: [SPARK-28077][SQL] Support ANSI SQL 
OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506610229
 
 
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24918: [SPARK-28077][SQL] Support ANSI SQL 
OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506610233
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106984/
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
SparkQA removed a comment on issue #24918: [SPARK-28077][SQL] Support ANSI SQL 
OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506578525
 
 
   **[Test build #106984 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106984/testReport)**
 for PR 24918 at commit 
[`2a9334a`](https://github.com/apache/spark/commit/2a9334a7d3fb71df749f7d431375385cd0bf1cfd).


This is an automated message from the 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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
SparkQA commented on issue #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY 
function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506609870
 
 
   **[Test build #106984 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106984/testReport)**
 for PR 24918 at commit 
[`2a9334a`](https://github.com/apache/spark/commit/2a9334a7d3fb71df749f7d431375385cd0bf1cfd).
* 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 #24043: [SPARK-11412][SQL] Support merge schema for ORC

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24043: [SPARK-11412][SQL] Support 
merge schema for ORC
URL: https://github.com/apache/spark/pull/24043#issuecomment-506607245
 
 
   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 #24043: [SPARK-11412][SQL] Support merge schema for ORC

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24043: [SPARK-11412][SQL] Support 
merge schema for ORC
URL: https://github.com/apache/spark/pull/24043#issuecomment-506607248
 
 
   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/12187/
   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 #24043: [SPARK-11412][SQL] Support merge schema for ORC

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24043: [SPARK-11412][SQL] Support merge 
schema for ORC
URL: https://github.com/apache/spark/pull/24043#issuecomment-506607248
 
 
   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/12187/
   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 #24043: [SPARK-11412][SQL] Support merge schema for ORC

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24043: [SPARK-11412][SQL] Support merge 
schema for ORC
URL: https://github.com/apache/spark/pull/24043#issuecomment-506607245
 
 
   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 #24043: [SPARK-11412][SQL] Support merge schema for ORC

2019-06-27 Thread GitBox
SparkQA commented on issue #24043: [SPARK-11412][SQL] Support merge schema for 
ORC
URL: https://github.com/apache/spark/pull/24043#issuecomment-506606136
 
 
   **[Test build #106987 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106987/testReport)**
 for PR 24043 at commit 
[`a6fc2d0`](https://github.com/apache/spark/commit/a6fc2d0d3b542c402e426ff125ff42822ddb4b7c).


This is an automated message from the 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] gengliangwang commented on issue #24043: [SPARK-11412][SQL] Support merge schema for ORC

2019-06-27 Thread GitBox
gengliangwang commented on issue #24043: [SPARK-11412][SQL] Support merge 
schema for ORC
URL: https://github.com/apache/spark/pull/24043#issuecomment-506605934
 
 
   retest this please.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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] WangGuangxin commented on issue #24043: [SPARK-11412][SQL] Support merge schema for ORC

2019-06-27 Thread GitBox
WangGuangxin commented on issue #24043: [SPARK-11412][SQL] Support merge schema 
for ORC
URL: https://github.com/apache/spark/pull/24043#issuecomment-506605111
 
 
   > Retest it please
   
   @gengliangwang Could you please make jenkins retest this? Thanks


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
SparkQA commented on issue #24986: [SPARK-28185][ML][SQL] Trigger pandas 
iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506604697
 
 
   **[Test build #106986 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106986/testReport)**
 for PR 24986 at commit 
[`d2aa561`](https://github.com/apache/spark/commit/d2aa561130d994b6cde4f915af547c6fb9330cc9).


This is an automated message from the 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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24986: [SPARK-28185][ML][SQL] Trigger 
pandas iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506604271
 
 
   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/12186/
   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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24986: [SPARK-28185][ML][SQL] Trigger 
pandas iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506604266
 
 
   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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24986: [SPARK-28185][ML][SQL] Trigger pandas 
iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506604271
 
 
   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/12186/
   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 #24986: [SPARK-28185][ML][SQL] Trigger pandas iterator UDF closing stuff when iterator stop early

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24986: [SPARK-28185][ML][SQL] Trigger pandas 
iterator UDF closing stuff when iterator stop early
URL: https://github.com/apache/spark/pull/24986#issuecomment-506604266
 
 
   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] AngersZhuuuu removed a comment on issue #24973: [SPARK-28169] Fix Partition table partition PushDown failed by "OR" expression

2019-06-27 Thread GitBox
AngersZh removed a comment on issue #24973: [SPARK-28169] Fix Partition 
table partition  PushDown failed by "OR" expression 
URL: https://github.com/apache/spark/pull/24973#issuecomment-506584232
 
 
   @cloud-fan  
   Since I am familiar with code about hive , I just looked about other 
strategies about  scan data. 
   How about we convert filter expression set to CNF in  PhysicalOperation. 
Then change the way get partition expressions like below, since with CNF, just 
use **subSet** to judge is not enough:
   ` val partitionKeyFilters = `
   `  ExpressionSet(normalizedFilters`
  ` .filterNot(SubqueryExpression.hasSubquery(_))`
   `.filter(_.references.subsetOf(partitionSet))`


This is an automated message from the 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 #24995: [SPARK-28196][SQL] Add a new `listTables` and `listLocalTempViews` APIs for SessionCatalog

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24995: [SPARK-28196][SQL] Add a new 
`listTables` and `listLocalTempViews` APIs for SessionCatalog
URL: https://github.com/apache/spark/pull/24995#issuecomment-506600016
 
 
   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/12185/
   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 #24995: [SPARK-28196][SQL] Add a new `listTables` and `listLocalTempViews` APIs for SessionCatalog

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24995: [SPARK-28196][SQL] Add a new 
`listTables` and `listLocalTempViews` APIs for SessionCatalog
URL: https://github.com/apache/spark/pull/24995#issuecomment-50669
 
 
   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 #24995: [SPARK-28196][SQL] Add a new `listTables` and `listLocalTempViews` APIs for SessionCatalog

2019-06-27 Thread GitBox
SparkQA commented on issue #24995: [SPARK-28196][SQL] Add a new `listTables` 
and `listLocalTempViews` APIs for SessionCatalog
URL: https://github.com/apache/spark/pull/24995#issuecomment-506600384
 
 
   **[Test build #106985 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106985/testReport)**
 for PR 24995 at commit 
[`df4cb47`](https://github.com/apache/spark/commit/df4cb471c9712a2fe496664028d9303caebd8777).


This is an automated message from the 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 #24995: [SPARK-28196][SQL] Add a new `listTables` and `listLocalTempViews` APIs for SessionCatalog

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24995: [SPARK-28196][SQL] Add a new 
`listTables` and `listLocalTempViews` APIs for SessionCatalog
URL: https://github.com/apache/spark/pull/24995#issuecomment-50669
 
 
   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 #24995: [SPARK-28196][SQL] Add a new `listTables` and `listLocalTempViews` APIs for SessionCatalog

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24995: [SPARK-28196][SQL] Add a new 
`listTables` and `listLocalTempViews` APIs for SessionCatalog
URL: https://github.com/apache/spark/pull/24995#issuecomment-506600016
 
 
   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/12185/
   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] wangyum opened a new pull request #24995: [SPARK-28196][SQL] Add a new `listTables` and `listLocalTempViews` APIs for SessionCatalog

2019-06-27 Thread GitBox
wangyum opened a new pull request #24995: [SPARK-28196][SQL] Add a new 
`listTables` and `listLocalTempViews` APIs for SessionCatalog
URL: https://github.com/apache/spark/pull/24995
 
 
   ## What changes were proposed in this pull request?
   
   This pr add two API for 
[SessionCatalog](https://github.com/apache/spark/blob/df4cb471c9712a2fe496664028d9303caebd8777/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala):
   ```scala
   def listTables(db: String, pattern: String, includeLocalTempViews: Boolean): 
Seq[TableIdentifier]
   
   def listLocalTempViews(pattern: String): Seq[TableIdentifier]
   ```
   Because in some cases `listTables` does not need local temporary view and 
sometimes only need list local temporary view.
   
   ## How was this patch tested?
   
   unit tests


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #24994: [SPARK-28133] Adding inverse hyperbolic functions in SQL

2019-06-27 Thread GitBox
HyukjinKwon commented on a change in pull request #24994: [SPARK-28133] Adding 
inverse hyperbolic functions in SQL
URL: https://github.com/apache/spark/pull/24994#discussion_r298451886
 
 

 ##
 File path: python/pyspark/sql/functions.py
 ##
 @@ -187,6 +187,15 @@ def _():
' column name, and null values appear after non-null 
values',
 }
 
+_functions_3_0_over_column = {
+'asinh': """:param col: hyperbolic angle
 
 Review comment:
   Can we avoid to add them per 
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L42-L56?


This is an automated message from the 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] LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add a filter interface to KVStore to speed up the entities retrieve

2019-06-27 Thread GitBox
LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add 
a filter interface to KVStore to speed up the entities retrieve
URL: https://github.com/apache/spark/pull/24982#discussion_r298449439
 
 

 ##
 File path: 
common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java
 ##
 @@ -197,6 +201,23 @@ public void delete(Class type, Object naturalKey) 
throws Exception {
 };
   }
 
+  @Override
+  public  KVStoreView viewWithCondition(Class type, Predicate 
condition) throws Exception {
+return new KVStoreView() {
+  @Override
+  public Iterator iterator() {
+try {
+  Iterator base = new LevelDBIterator<>(type, LevelDB.this, this);
+  Iterable iterable = () -> base;
 
 Review comment:
   
https://stackoverflow.com/questions/24511052/how-to-convert-an-iterator-to-a-stream


This is an automated message from the 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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a 
global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506595347
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106983/
   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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24530: [SPARK-27520][CORE][WIP] 
Introduce a global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506595343
 
 
   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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24530: [SPARK-27520][CORE][WIP] 
Introduce a global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506595347
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106983/
   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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a 
global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506595343
 
 
   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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
SparkQA removed a comment on issue #24530: [SPARK-27520][CORE][WIP] Introduce a 
global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506573221
 
 
   **[Test build #106983 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106983/testReport)**
 for PR 24530 at commit 
[`9946fb7`](https://github.com/apache/spark/commit/9946fb7cf85d66f44dc1cdc510d8be435fdd423e).


This is an automated message from the 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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
SparkQA commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a global 
config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506595054
 
 
   **[Test build #106983 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106983/testReport)**
 for PR 24530 at commit 
[`9946fb7`](https://github.com/apache/spark/commit/9946fb7cf85d66f44dc1cdc510d8be435fdd423e).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds the following public classes _(experimental)_:
 * `class SparkHadoopConf(var conf: Configuration) `


This is an automated message from the 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] LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add a filter interface to KVStore to speed up the entities retrieve

2019-06-27 Thread GitBox
LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add 
a filter interface to KVStore to speed up the entities retrieve
URL: https://github.com/apache/spark/pull/24982#discussion_r298448559
 
 

 ##
 File path: 
common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java
 ##
 @@ -197,6 +201,23 @@ public void delete(Class type, Object naturalKey) 
throws Exception {
 };
   }
 
+  @Override
+  public  KVStoreView viewWithCondition(Class type, Predicate 
condition) throws Exception {
+return new KVStoreView() {
+  @Override
+  public Iterator iterator() {
+try {
+  Iterator base = new LevelDBIterator<>(type, LevelDB.this, this);
+  Iterable iterable = () -> base;
 
 Review comment:
   > you need to return a new iterator each time
   
   Only `LevelDBIterator` could be accepted here since in 
`KVStoreView.closeableIterator()`, it casts the iterator to `KVStoreIterator`:
   ```java
 public KVStoreIterator closeableIterator() throws Exception {
   return (KVStoreIterator) iterator();
 }
   ```
   So I use a stream API instead of new a iterator. Do I misunderstand here?


This is an automated message from the 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 #24994: [SPARK-28133] Adding inverse hyperbolic functions in SQL

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24994: [SPARK-28133] Adding inverse 
hyperbolic functions in SQL
URL: https://github.com/apache/spark/pull/24994#issuecomment-506592593
 
 
   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] AmplabJenkins commented on issue #24994: [SPARK-28133] Adding inverse hyperbolic functions in SQL

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24994: [SPARK-28133] Adding inverse 
hyperbolic functions in SQL
URL: https://github.com/apache/spark/pull/24994#issuecomment-506592921
 
 
   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] AmplabJenkins removed a comment on issue #24994: [SPARK-28133] Adding inverse hyperbolic functions in SQL

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24994: [SPARK-28133] Adding inverse 
hyperbolic functions in SQL
URL: https://github.com/apache/spark/pull/24994#issuecomment-506592506
 
 
   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] AmplabJenkins commented on issue #24994: [SPARK-28133] Adding inverse hyperbolic functions in SQL

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24994: [SPARK-28133] Adding inverse 
hyperbolic functions in SQL
URL: https://github.com/apache/spark/pull/24994#issuecomment-506592506
 
 
   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] AmplabJenkins commented on issue #24994: [SPARK-28133] Adding inverse hyperbolic functions in SQL

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24994: [SPARK-28133] Adding inverse 
hyperbolic functions in SQL
URL: https://github.com/apache/spark/pull/24994#issuecomment-506592593
 
 
   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] Tonix517 opened a new pull request #24994: [SPARK-28133] Adding inverse hyperbolic functions in SQL

2019-06-27 Thread GitBox
Tonix517 opened a new pull request #24994: [SPARK-28133] Adding inverse 
hyperbolic functions in SQL
URL: https://github.com/apache/spark/pull/24994
 
 
   ## What changes were proposed in this pull request?
   
   Inverse hyperbolic functions (asinh, acosh, atanh) were missing from SQL. 
Now we add support to those functions.
   
   ## How was this patch tested?
   
   Added new tests.
   ./build/sbt "testOnly 
org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite
   ./build/sbt "testOnly org.apache.spark.sql.MathFunctionsSuite
   
   [In spark-shell] 
   spark.sql("select asinh(xx)").show
   spark.sql("select acosh(xx)").show
   spark.sql("select atanh(xx)").show
   
   Please review https://spark.apache.org/contributing.html before opening a 
pull request.
   
   @wangyum @mgaido91 


This is an automated message from the 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 #24993: [SPARK-18299][SQL] Allow more aggregations on KeyValueGroupedDataset

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24993: [SPARK-18299][SQL] Allow more 
aggregations on KeyValueGroupedDataset
URL: https://github.com/apache/spark/pull/24993#issuecomment-506590354
 
 
   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] AmplabJenkins commented on issue #24993: [SPARK-18299][SQL] Allow more aggregations on KeyValueGroupedDataset

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24993: [SPARK-18299][SQL] Allow more 
aggregations on KeyValueGroupedDataset
URL: https://github.com/apache/spark/pull/24993#issuecomment-506590669
 
 
   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] AmplabJenkins removed a comment on issue #24993: [SPARK-18299][SQL] Allow more aggregations on KeyValueGroupedDataset

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24993: [SPARK-18299][SQL] Allow more 
aggregations on KeyValueGroupedDataset
URL: https://github.com/apache/spark/pull/24993#issuecomment-506590278
 
 
   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] AmplabJenkins commented on issue #24993: [SPARK-18299][SQL] Allow more aggregations on KeyValueGroupedDataset

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24993: [SPARK-18299][SQL] Allow more 
aggregations on KeyValueGroupedDataset
URL: https://github.com/apache/spark/pull/24993#issuecomment-506590354
 
 
   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] AmplabJenkins commented on issue #24993: [SPARK-18299][SQL] Allow more aggregations on KeyValueGroupedDataset

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24993: [SPARK-18299][SQL] Allow more 
aggregations on KeyValueGroupedDataset
URL: https://github.com/apache/spark/pull/24993#issuecomment-506590278
 
 
   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] nooberfsh opened a new pull request #24993: [SPARK-18299][SQL] Allow more aggregations on KeyValueGroupedDataset

2019-06-27 Thread GitBox
nooberfsh opened a new pull request #24993: [SPARK-18299][SQL] Allow more 
aggregations on KeyValueGroupedDataset
URL: https://github.com/apache/spark/pull/24993
 
 
   ## What changes were proposed in this pull request?
   
   Add 4 additional agg to KeyValueGroupedDataset
   
   ## How was this patch tested?
   
   New test in DatasetSuite for typed aggregation
   


This is an automated message from the 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 #22282: [SPARK-23539][SS] Add support for Kafka headers in Structured Streaming

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #22282: [SPARK-23539][SS] Add support 
for Kafka headers in Structured Streaming
URL: https://github.com/apache/spark/pull/22282#issuecomment-506587817
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106982/
   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 #22282: [SPARK-23539][SS] Add support for Kafka headers in Structured Streaming

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #22282: [SPARK-23539][SS] Add support 
for Kafka headers in Structured Streaming
URL: https://github.com/apache/spark/pull/22282#issuecomment-506587815
 
 
   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 #22282: [SPARK-23539][SS] Add support for Kafka headers in Structured Streaming

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #22282: [SPARK-23539][SS] Add support for 
Kafka headers in Structured Streaming
URL: https://github.com/apache/spark/pull/22282#issuecomment-506587815
 
 
   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 #22282: [SPARK-23539][SS] Add support for Kafka headers in Structured Streaming

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #22282: [SPARK-23539][SS] Add support for 
Kafka headers in Structured Streaming
URL: https://github.com/apache/spark/pull/22282#issuecomment-506587817
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106982/
   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 #22282: [SPARK-23539][SS] Add support for Kafka headers in Structured Streaming

2019-06-27 Thread GitBox
SparkQA commented on issue #22282: [SPARK-23539][SS] Add support for Kafka 
headers in Structured Streaming
URL: https://github.com/apache/spark/pull/22282#issuecomment-506587554
 
 
   **[Test build #106982 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106982/testReport)**
 for PR 22282 at commit 
[`7988687`](https://github.com/apache/spark/commit/79886876744675b1a38b0c7bbf286ad1f49c0ceb).
* 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 #22282: [SPARK-23539][SS] Add support for Kafka headers in Structured Streaming

2019-06-27 Thread GitBox
SparkQA removed a comment on issue #22282: [SPARK-23539][SS] Add support for 
Kafka headers in Structured Streaming
URL: https://github.com/apache/spark/pull/22282#issuecomment-506557340
 
 
   **[Test build #106982 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106982/testReport)**
 for PR 22282 at commit 
[`7988687`](https://github.com/apache/spark/commit/79886876744675b1a38b0c7bbf286ad1f49c0ceb).


This is an automated message from the 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 #24992: [SPARK-28194][SQL] Refactor code to prevent None.get

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24992: [SPARK-28194][SQL] Refactor 
code to prevent None.get
URL: https://github.com/apache/spark/pull/24992#issuecomment-506584469
 
 
   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] AmplabJenkins commented on issue #24992: [SPARK-28194][SQL] Refactor code to prevent None.get

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24992: [SPARK-28194][SQL] Refactor code to 
prevent None.get
URL: https://github.com/apache/spark/pull/24992#issuecomment-506584824
 
 
   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] AmplabJenkins removed a comment on issue #24992: [SPARK-28194][SQL] Refactor code to prevent None.get

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24992: [SPARK-28194][SQL] Refactor 
code to prevent None.get
URL: https://github.com/apache/spark/pull/24992#issuecomment-506584376
 
 
   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] AmplabJenkins commented on issue #24992: [SPARK-28194][SQL] Refactor code to prevent None.get

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24992: [SPARK-28194][SQL] Refactor code to 
prevent None.get
URL: https://github.com/apache/spark/pull/24992#issuecomment-506584469
 
 
   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] AmplabJenkins commented on issue #24992: [SPARK-28194][SQL] Refactor code to prevent None.get

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24992: [SPARK-28194][SQL] Refactor code to 
prevent None.get
URL: https://github.com/apache/spark/pull/24992#issuecomment-506584376
 
 
   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] AngersZhuuuu commented on issue #24973: [SPARK-28169] Fix Partition table partition PushDown failed by "OR" expression

2019-06-27 Thread GitBox
AngersZh commented on issue #24973: [SPARK-28169] Fix Partition table 
partition  PushDown failed by "OR" expression 
URL: https://github.com/apache/spark/pull/24973#issuecomment-506584232
 
 
   @cloud-fan  
   Since I am familiar with code about hive , I just looked about other 
strategies about  scan data. 
   How about we convert filter expression set to CNF in  PhysicalOperation. 
Then change the way get partition expressions like below, since with CNF, just 
use **subSet** to judge is not enough:
   ` val partitionKeyFilters = `
   `  ExpressionSet(normalizedFilters`
  ` .filterNot(SubqueryExpression.hasSubquery(_))`
   `.filter(_.references.subsetOf(partitionSet))`


This is an automated message from the 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] turboFei opened a new pull request #24992: [SPARK-28194][SQL] Refactor code to prevent None.get

2019-06-27 Thread GitBox
turboFei opened a new pull request #24992: [SPARK-28194][SQL] Refactor code to 
prevent None.get
URL: https://github.com/apache/spark/pull/24992
 
 
   ## What changes were proposed in this pull request?
   We have meet a NoSuchElementException during the phase of EnsureRequirement.
   ```
   java.util.NoSuchElementException: None.get
at scala.None$.get(Option.scala:347)
at scala.None$.get(Option.scala:345)
at 
org.apache.spark.sql.execution.exchange.EnsureRequirements$$anonfun$reorder$1.apply(EnsureRequirements.scala:239)
at 
org.apache.spark.sql.execution.exchange.EnsureRequirements$$anonfun$reorder$1.apply(EnsureRequirements.scala:234)
at scala.collection.immutable.List.foreach(List.scala:381)
at 
org.apache.spark.sql.execution.exchange.EnsureRequirements.reorder(EnsureRequirements.scala:234)
at 
org.apache.spark.sql.execution.exchange.EnsureRequirements.reorderJoinKeys(EnsureRequirements.scala:257)
at 
org.apache.spark.sql.execution.exchange.EnsureRequirements.org$apache$spark$sql$execution$exchange$EnsureRequirements$$reorderJoinPredicates(EnsureRequirements.scala:297)
at 
org.apache.spark.sql.execution.exchange.EnsureRequirements$$anonfun$apply$1.applyOrElse(EnsureRequirements.scala:312)
at 
org.apache.spark.sql.execution.exchange.EnsureRequirements$$anonfun$apply$1.applyOrElse(EnsureRequirements.scala:304)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$2.apply(TreeNode.scala:293)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$2.apply(TreeNode.scala:293)
at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:292)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:286)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:286)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:286)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:286)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:286)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:286)
at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:286)
   ```
   In this PR, I refactor the code to prevent None.get.
   
   ## How was this patch tested?
   
   Existing Unit 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] AmplabJenkins removed a comment on issue #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24918: [SPARK-28077][SQL] Support 
ANSI SQL OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506579483
 
 
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24918: [SPARK-28077][SQL] Support 
ANSI SQL OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506579490
 
 
   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/12184/
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24918: [SPARK-28077][SQL] Support ANSI SQL 
OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506579490
 
 
   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/12184/
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24918: [SPARK-28077][SQL] Support ANSI SQL 
OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506579483
 
 
   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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
SparkQA commented on issue #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY 
function.
URL: https://github.com/apache/spark/pull/24918#issuecomment-506578525
 
 
   **[Test build #106984 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106984/testReport)**
 for PR 24918 at commit 
[`2a9334a`](https://github.com/apache/spark/commit/2a9334a7d3fb71df749f7d431375385cd0bf1cfd).


This is an automated message from the 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] gengliangwang commented on issue #24849: [SPARK-28018][SQL] Allow upcasting decimal to double/float

2019-06-27 Thread GitBox
gengliangwang commented on issue #24849: [SPARK-28018][SQL] Allow upcasting 
decimal to double/float
URL: https://github.com/apache/spark/pull/24849#issuecomment-506578483
 
 
   > This is not a safe cast.
   
   I think it is explainable. There is no precision loss in the casting itself
   
   > In the DSv2 sync we discussed options that would work. Why not go with 
those?
   
   As I remember, we didn't have a solution in DSv2 sync and we decide to come 
up with solutions offline.


This is an automated message from the 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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
beliefer commented on a change in pull request #24918: [SPARK-28077][SQL] 
Support ANSI SQL OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#discussion_r298434757
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
 ##
 @@ -454,6 +455,64 @@ case class StringReplace(srcExpr: Expression, searchExpr: 
Expression, replaceExp
   override def prettyName: String = "replace"
 }
 
+object Overlay {
+
+  def calculate(input: UTF8String, replace: UTF8String, pos: Int, len: Int): 
UTF8String = {
+val builder = new UTF8StringBuilder
+builder.append(input.substringSQL(1, pos - 1))
+builder.append(replace)
+var length = len
+if (len < 0) {
+  length = replace.toString().length()
+}
+builder.append(input.substringSQL(pos + length, Int.MaxValue))
+builder.build()
+  }
+}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = "_FUNC_(input, replace, pos[, len]) - Replace `input` with `replace` 
that starts at `pos` and is of length `len`.",
+  examples = """
+Examples:
+  > SELECT _FUNC_('Spark SQL' PLACING '_' FROM 6);
+   Spark_SQL
+  > SELECT _FUNC_('Spark SQL' PLACING 'CORE' FROM 7);
+   Spark CORE
+  > SELECT _FUNC_('Spark SQL' PLACING 'ANSI ' FROM 7 FOR 0);
+   Spark ANSI SQL
+  > SELECT _FUNC_('Spark SQL' PLACING 'tructured' FROM 2 FOR 4);
+   Structured SQL
+  """)
+// scalastyle:on line.size.limit
+case class Overlay(input: Expression, replace: Expression, pos: Expression, 
len: Expression)
+  extends QuaternaryExpression with ImplicitCastInputTypes with NullIntolerant 
{
+
+  def this(str: Expression, replace: Expression, pos: Expression) = {
+this(str, replace, pos, Literal.create(-1, IntegerType))
+  }
+
+  override def dataType: DataType = StringType
+
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(StringType, StringType, IntegerType, IntegerType)
+
+  override def children: Seq[Expression] = input :: replace :: pos :: len :: 
Nil
+
+  override def nullSafeEval(inputEval: Any, replaceEval: Any, posEval: Any, 
lenEval: Any): Any = {
+val inputStr = inputEval.asInstanceOf[UTF8String]
+val replaceStr = replaceEval.asInstanceOf[UTF8String]
+val position = posEval.asInstanceOf[Int]
+val length = lenEval.asInstanceOf[Int]
+Overlay.calculate(inputStr, replaceStr, position, length)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+defineCodeGen(ctx, ev, (input, replace, pos, len) => 
"org.apache.spark.sql.catalyst." +
 
 Review comment:
   OK. I changed 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] beliefer commented on a change in pull request #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
beliefer commented on a change in pull request #24918: [SPARK-28077][SQL] 
Support ANSI SQL OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#discussion_r298434725
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
 ##
 @@ -454,6 +455,64 @@ case class StringReplace(srcExpr: Expression, searchExpr: 
Expression, replaceExp
   override def prettyName: String = "replace"
 }
 
+object Overlay {
+
+  def calculate(input: UTF8String, replace: UTF8String, pos: Int, len: Int): 
UTF8String = {
+val builder = new UTF8StringBuilder
+builder.append(input.substringSQL(1, pos - 1))
+builder.append(replace)
+var length = len
+if (len < 0) {
+  length = replace.toString().length()
 
 Review comment:
   OK. I learned 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] cloud-fan commented on issue #24960: [SPARK-28156][SQL] Self-join should not miss cached view

2019-06-27 Thread GitBox
cloud-fan commented on issue #24960: [SPARK-28156][SQL] Self-join should not 
miss cached view
URL: https://github.com/apache/spark/pull/24960#issuecomment-506578188
 
 
   Then we should revisit https://github.com/apache/spark/pull/24236 which adds 
Project to do dedup for Union. I don't know if there are more places like this.
   
   The cache manager uses `QueryPlan.sameResult` to do cache look up, and it 
looks weird if an extra no-op project breaks `QueryPlan.sameResult`. Shall we 
update `Project.canonicalized` to remove no-op project node?
   


This is an automated message from the 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] LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add a filter interface to KVStore to speed up the entities retrieve

2019-06-27 Thread GitBox
LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add 
a filter interface to KVStore to speed up the entities retrieve
URL: https://github.com/apache/spark/pull/24982#discussion_r298431375
 
 

 ##
 File path: 
common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##
 @@ -106,6 +106,12 @@ public void delete(Class type, Object naturalKey) {
 return list != null ? list.view() : emptyView();
   }
 
+  @Override
+  public  KVStoreView viewWithCondition(Class type, Predicate 
condition) {
+InstanceList list = inMemoryLists.get(type);
 
 Review comment:
   Do you mean dedup like this?
   ```scala
 @Override
 public  KVStoreView view(Class type){
   return viewWithCondition(type, t -> true);
 }
   ```
   We cannot invoke `view` in `viewWithCondition` since `list.view(type)` will 
create much more instances than `list.viewWithCondition` indeed.


This is an automated message from the 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] LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add a filter interface to KVStore to speed up the entities retrieve

2019-06-27 Thread GitBox
LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add 
a filter interface to KVStore to speed up the entities retrieve
URL: https://github.com/apache/spark/pull/24982#discussion_r298431375
 
 

 ##
 File path: 
common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##
 @@ -106,6 +106,12 @@ public void delete(Class type, Object naturalKey) {
 return list != null ? list.view() : emptyView();
   }
 
+  @Override
+  public  KVStoreView viewWithCondition(Class type, Predicate 
condition) {
+InstanceList list = inMemoryLists.get(type);
 
 Review comment:
   Do you mean dedup like this?
   ```scala
 @Override
 public  KVStoreView view(Class type){
   return viewWithCondition(type, t -> true);
 }
   ```
   It cannot invoke `view` in `viewWithCondition` since `list.view(type)` will 
create much more instances than `list.viewWithCondition` indeed.


This is an automated message from the 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] cloud-fan commented on issue #24973: [SPARK-28169] Fix Partition table partition PushDown failed by "OR" expression

2019-06-27 Thread GitBox
cloud-fan commented on issue #24973: [SPARK-28169] Fix Partition table 
partition  PushDown failed by "OR" expression 
URL: https://github.com/apache/spark/pull/24973#issuecomment-506576452
 
 
   This place is definitely not the only place that extracts partition 
predicates (e.g. `FileSourceStrategy`), I'm -1 to add a hack just in this 
place. I still prefer to add a CNF conversion rule to solve this problem for 
all the places, or other general solutions.


This is an automated message from the 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] cloud-fan commented on issue #24849: [SPARK-28018][SQL] Allow upcasting decimal to double/float

2019-06-27 Thread GitBox
cloud-fan commented on issue #24849: [SPARK-28018][SQL] Allow upcasting decimal 
to double/float
URL: https://github.com/apache/spark/pull/24849#issuecomment-506575380
 
 
   > the problem is with implicit casts inserted by Spark, not explicit casts 
included in the query.
   
   @rdblue seems you misread the query.
   
   `insert into test_spark_27856 values(cast(112345678901234567890.123456 as 
decimal(31,10)))`. We can replace the cast with a decimal literal to be more 
clear. It inserts a decimal value to a float column, so implicit cast does 
happen.


This is an automated message from the 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] cloud-fan commented on a change in pull request #24978: [SPARK-28177][SQL] Adjust post shuffle partition number in adaptive execution

2019-06-27 Thread GitBox
cloud-fan commented on a change in pull request #24978: [SPARK-28177][SQL] 
Adjust post shuffle partition number in adaptive execution
URL: https://github.com/apache/spark/pull/24978#discussion_r298431930
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala
 ##
 @@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.adaptive.rule
+
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration.Duration
+
+import org.apache.spark.MapOutputStatistics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, 
UnknownPartitioning}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, 
UnaryExecNode}
+import org.apache.spark.sql.execution.adaptive.{QueryStageExec, 
ReusedQueryStageExec, ShuffleQueryStageExec}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * A rule to adjust the post shuffle partitions based on the map output 
statistics.
+ *
+ * The strategy used to determine the number of post-shuffle partitions is 
described as follows.
+ * To determine the number of post-shuffle partitions, we have a target input 
size for a
+ * post-shuffle partition. Once we have size statistics of all pre-shuffle 
partitions, we will do
+ * a pass of those statistics and pack pre-shuffle partitions with continuous 
indices to a single
+ * post-shuffle partition until adding another pre-shuffle partition would 
cause the size of a
+ * post-shuffle partition to be greater than the target size.
+ *
+ * For example, we have two stages with the following pre-shuffle partition 
size statistics:
+ * stage 1: [100 MiB, 20 MiB, 100 MiB, 10MiB, 30 MiB]
+ * stage 2: [10 MiB,  10 MiB, 70 MiB,  5 MiB, 5 MiB]
+ * assuming the target input size is 128 MiB, we will have four post-shuffle 
partitions,
+ * which are:
+ *  - post-shuffle partition 0: pre-shuffle partition 0 (size 110 MiB)
+ *  - post-shuffle partition 1: pre-shuffle partition 1 (size 30 MiB)
+ *  - post-shuffle partition 2: pre-shuffle partition 2 (size 170 MiB)
+ *  - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MiB)
+ */
+case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] {
+
+  override def apply(plan: SparkPlan): SparkPlan = {
+val shuffleMetrics: Seq[MapOutputStatistics] = plan.collect {
+  case stage: ShuffleQueryStageExec =>
+val metricsFuture = stage.mapOutputStatisticsFuture
+assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should 
already be ready")
+ThreadUtils.awaitResult(metricsFuture, Duration.Zero)
+  case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) =>
+val metricsFuture = stage.mapOutputStatisticsFuture
+assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should 
already be ready")
+ThreadUtils.awaitResult(metricsFuture, Duration.Zero)
+}
+
+if (!plan.collectLeaves().forall(_.isInstanceOf[QueryStageExec])) {
 
 Review comment:
   Obviously we don't want to adjust num shuffle partitions if there is no 
shuffle in this stage, so +1 to change the condition to `isAllQueryStage && 
shuffleStageCount > 0`.


This is an automated message from the 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] LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add a filter interface to KVStore to speed up the entities retrieve

2019-06-27 Thread GitBox
LantaoJin commented on a change in pull request #24982: [SPARK-28181][CORE] Add 
a filter interface to KVStore to speed up the entities retrieve
URL: https://github.com/apache/spark/pull/24982#discussion_r298431375
 
 

 ##
 File path: 
common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##
 @@ -106,6 +106,12 @@ public void delete(Class type, Object naturalKey) {
 return list != null ? list.view() : emptyView();
   }
 
+  @Override
+  public  KVStoreView viewWithCondition(Class type, Predicate 
condition) {
+InstanceList list = inMemoryLists.get(type);
 
 Review comment:
   Oh, 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] SparkQA commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
SparkQA commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a global 
config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506573221
 
 
   **[Test build #106983 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/106983/testReport)**
 for PR 24530 at commit 
[`9946fb7`](https://github.com/apache/spark/commit/9946fb7cf85d66f44dc1cdc510d8be435fdd423e).


This is an automated message from the 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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24530: [SPARK-27520][CORE][WIP] 
Introduce a global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506572898
 
 
   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/12183/
   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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24530: [SPARK-27520][CORE][WIP] 
Introduce a global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506572896
 
 
   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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a 
global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506572898
 
 
   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/12183/
   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 #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
AmplabJenkins commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a 
global config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506572896
 
 
   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] Ngone51 commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a global config system to replace hadoopConfiguration

2019-06-27 Thread GitBox
Ngone51 commented on issue #24530: [SPARK-27520][CORE][WIP] Introduce a global 
config system to replace hadoopConfiguration
URL: https://github.com/apache/spark/pull/24530#issuecomment-506572394
 
 
   Jenkins, retest this please.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #24918: [SPARK-28077][SQL] Support ANSI SQL OVERLAY function.

2019-06-27 Thread GitBox
beliefer commented on a change in pull request #24918: [SPARK-28077][SQL] 
Support ANSI SQL OVERLAY function.
URL: https://github.com/apache/spark/pull/24918#discussion_r298427889
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
 ##
 @@ -454,6 +454,66 @@ case class StringReplace(srcExpr: Expression, searchExpr: 
Expression, replaceExp
   override def prettyName: String = "replace"
 }
 
+object Overlay {
+
+  def calcuate(input: UTF8String, replace: UTF8String, pos: Integer, len: 
Integer): UTF8String = {
+val header = input.substringSQL(1, pos - 1)
+var length = len
+if (len < 0) {
+  length = replace.toString().length()
+}
+val tailer = input.substringSQL(pos + length, Int.MaxValue)
+UTF8String.fromString(header.toString + replace.toString + tailer.toString)
+  }
+}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = "_FUNC_(input, replace, pos[, len]) - Replace `input` with `replace` 
that starts at `pos` and is of length `len`.",
+  examples = """
+Examples:
+  > SELECT _FUNC_('Spark SQL' PLACING '_' FROM 6);
+   Spark_SQL
+  > SELECT _FUNC_('Spark SQL' PLACING 'CORE' FROM 7);
+   Spark CORE
+  > SELECT _FUNC_('Spark SQL' PLACING 'ANSI ' FROM 7 FOR 0);
+   Spark ANSI SQL
+  > SELECT _FUNC_('Spark SQL' PLACING 'tructured' FROM 2 FOR 4);
+   Structured SQL
+  """)
+// scalastyle:on line.size.limit
+case class Overlay(input: Expression, replace: Expression, pos: Expression, 
len: Expression)
+  extends QuaternaryExpression with ImplicitCastInputTypes with NullIntolerant 
{
+
+  def this(str: Expression, replace: Expression, pos: Expression) = {
+this(str, replace, pos, Literal.create(-1, IntegerType))
+  }
+
+  override def dataType: DataType = StringType
+
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(StringType, StringType, IntegerType, IntegerType)
+
+  override def children: Seq[Expression] = input :: replace :: pos :: len :: 
Nil
+
+  override def nullSafeEval(inputEval: Any, replaceEval: Any, posEval: Any, 
lenEval: Any): Any = {
+val inputStr = inputEval.asInstanceOf[UTF8String]
+val replaceStr = replaceEval.asInstanceOf[UTF8String]
+val position = posEval.asInstanceOf[Int]
+val length = lenEval.asInstanceOf[Int]
+Overlay.calcuate(inputStr, replaceStr, position, length)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+nullSafeCodeGen(ctx, ev, (input, replace, pos, len) => {
 
 Review comment:
   Yes, changed.


This is an automated message from the 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 closed pull request #24979: [SPARK-28179][SQL] Avoid hard-coded config: spark.sql.globalTempDatabase

2019-06-27 Thread GitBox
HyukjinKwon closed pull request #24979: [SPARK-28179][SQL] Avoid hard-coded 
config: spark.sql.globalTempDatabase
URL: https://github.com/apache/spark/pull/24979
 
 
   


This is an automated message from the 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 #24979: [SPARK-28179][SQL] Avoid hard-coded config: spark.sql.globalTempDatabase

2019-06-27 Thread GitBox
HyukjinKwon commented on issue #24979: [SPARK-28179][SQL] Avoid hard-coded 
config: spark.sql.globalTempDatabase
URL: https://github.com/apache/spark/pull/24979#issuecomment-506568833
 
 
   Merged to master.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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] AngersZhuuuu removed a comment on issue #24973: [SPARK-28169] Fix Partition table partition PushDown failed by "OR" expression

2019-06-27 Thread GitBox
AngersZh removed a comment on issue #24973: [SPARK-28169] Fix Partition 
table partition  PushDown failed by "OR" expression 
URL: https://github.com/apache/spark/pull/24973#issuecomment-506404166
 
 
   > This part really isn't my area
   
   Hmm, sorry , I just recently pull request in  spark community, a lot is not 
clear.


This is an automated message from the 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 #24990: [SPARK-28191][SS] New data source - state - reader part

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24990: [SPARK-28191][SS] New data 
source - state - reader part
URL: https://github.com/apache/spark/pull/24990#issuecomment-506564906
 
 
   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 #24990: [SPARK-28191][SS] New data source - state - reader part

2019-06-27 Thread GitBox
AmplabJenkins removed a comment on issue #24990: [SPARK-28191][SS] New data 
source - state - reader part
URL: https://github.com/apache/spark/pull/24990#issuecomment-506564911
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/106979/
   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



  1   2   3   4   5   6   7   >