[GitHub] spark pull request #15539: [SPARK-17994] [SQL] Add back a file status cache ...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15539#discussion_r95073558
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.JavaConverters._
+
+import com.google.common.cache._
+import org.apache.hadoop.fs.{FileStatus, Path}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.{SerializableConfiguration, SizeEstimator}
+
+/**
+ * A cache of the leaf files of partition directories. We cache these 
files in order to speed
+ * up iterated queries over the same set of partitions. Otherwise, each 
query would have to
+ * hit remote storage in order to gather file statistics for physical 
planning.
+ *
+ * Each resolved catalog table has its own FileStatusCache. When the 
backing relation for the
+ * table is refreshed via refreshTable() or refreshByPath(), this cache 
will be invalidated.
+ */
+abstract class FileStatusCache {
+  /**
+   * @return the leaf files for the specified path from this cache, or 
None if not cached.
+   */
+  def getLeafFiles(path: Path): Option[Array[FileStatus]] = None
+
+  /**
+   * Saves the given set of leaf files for a path in this cache.
+   */
+  def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit
+
+  /**
+   * Invalidates all data held by this cache.
+   */
+  def invalidateAll(): Unit
+}
+
+object FileStatusCache {
+  private var sharedCache: SharedInMemoryCache = null
+
+  /**
+   * @return a new FileStatusCache based on session configuration. Cache 
memory quota is
+   * shared across all clients.
+   */
+  def newCache(session: SparkSession): FileStatusCache = {
+synchronized {
+  if (session.sqlContext.conf.filesourcePartitionPruning &&
+  session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) {
--- End diff --

Or can we make the cache session-specific? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15539: [SPARK-17994] [SQL] Add back a file status cache ...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15539#discussion_r95073551
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.JavaConverters._
+
+import com.google.common.cache._
+import org.apache.hadoop.fs.{FileStatus, Path}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.{SerializableConfiguration, SizeEstimator}
+
+/**
+ * A cache of the leaf files of partition directories. We cache these 
files in order to speed
+ * up iterated queries over the same set of partitions. Otherwise, each 
query would have to
+ * hit remote storage in order to gather file statistics for physical 
planning.
+ *
+ * Each resolved catalog table has its own FileStatusCache. When the 
backing relation for the
+ * table is refreshed via refreshTable() or refreshByPath(), this cache 
will be invalidated.
+ */
+abstract class FileStatusCache {
+  /**
+   * @return the leaf files for the specified path from this cache, or 
None if not cached.
+   */
+  def getLeafFiles(path: Path): Option[Array[FileStatus]] = None
+
+  /**
+   * Saves the given set of leaf files for a path in this cache.
+   */
+  def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit
+
+  /**
+   * Invalidates all data held by this cache.
+   */
+  def invalidateAll(): Unit
+}
+
+object FileStatusCache {
+  private var sharedCache: SharedInMemoryCache = null
+
+  /**
+   * @return a new FileStatusCache based on session configuration. Cache 
memory quota is
+   * shared across all clients.
+   */
+  def newCache(session: SparkSession): FileStatusCache = {
+synchronized {
+  if (session.sqlContext.conf.filesourcePartitionPruning &&
+  session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) {
--- End diff --

If we allow users to change it at runtime, users might get strange results 
due to the global sharing cache. Some sessions might skip cache to directly 
change the metastore, some sessions are still using the cached values. Even in 
a single session, we might still face similar issues if we allow users to 
change it at runtime. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15539: [SPARK-17994] [SQL] Add back a file status cache ...

2017-01-07 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15539#discussion_r95073488
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.JavaConverters._
+
+import com.google.common.cache._
+import org.apache.hadoop.fs.{FileStatus, Path}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.{SerializableConfiguration, SizeEstimator}
+
+/**
+ * A cache of the leaf files of partition directories. We cache these 
files in order to speed
+ * up iterated queries over the same set of partitions. Otherwise, each 
query would have to
+ * hit remote storage in order to gather file statistics for physical 
planning.
+ *
+ * Each resolved catalog table has its own FileStatusCache. When the 
backing relation for the
+ * table is refreshed via refreshTable() or refreshByPath(), this cache 
will be invalidated.
+ */
+abstract class FileStatusCache {
+  /**
+   * @return the leaf files for the specified path from this cache, or 
None if not cached.
+   */
+  def getLeafFiles(path: Path): Option[Array[FileStatus]] = None
+
+  /**
+   * Saves the given set of leaf files for a path in this cache.
+   */
+  def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit
+
+  /**
+   * Invalidates all data held by this cache.
+   */
+  def invalidateAll(): Unit
+}
+
+object FileStatusCache {
+  private var sharedCache: SharedInMemoryCache = null
+
+  /**
+   * @return a new FileStatusCache based on session configuration. Cache 
memory quota is
+   * shared across all clients.
+   */
+  def newCache(session: SparkSession): FileStatusCache = {
+synchronized {
+  if (session.sqlContext.conf.filesourcePartitionPruning &&
+  session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) {
--- End diff --

Sounds good, as long as you can still change it at runtime.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15539: [SPARK-17994] [SQL] Add back a file status cache ...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15539#discussion_r95073252
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.JavaConverters._
+
+import com.google.common.cache._
+import org.apache.hadoop.fs.{FileStatus, Path}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.{SerializableConfiguration, SizeEstimator}
+
+/**
+ * A cache of the leaf files of partition directories. We cache these 
files in order to speed
+ * up iterated queries over the same set of partitions. Otherwise, each 
query would have to
+ * hit remote storage in order to gather file statistics for physical 
planning.
+ *
+ * Each resolved catalog table has its own FileStatusCache. When the 
backing relation for the
+ * table is refreshed via refreshTable() or refreshByPath(), this cache 
will be invalidated.
+ */
+abstract class FileStatusCache {
+  /**
+   * @return the leaf files for the specified path from this cache, or 
None if not cached.
+   */
+  def getLeafFiles(path: Path): Option[Array[FileStatus]] = None
+
+  /**
+   * Saves the given set of leaf files for a path in this cache.
+   */
+  def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit
+
+  /**
+   * Invalidates all data held by this cache.
+   */
+  def invalidateAll(): Unit
+}
+
+object FileStatusCache {
+  private var sharedCache: SharedInMemoryCache = null
+
+  /**
+   * @return a new FileStatusCache based on session configuration. Cache 
memory quota is
+   * shared across all clients.
+   */
+  def newCache(session: SparkSession): FileStatusCache = {
+synchronized {
+  if (session.sqlContext.conf.filesourcePartitionPruning &&
+  session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) {
--- End diff --

These two conf have to be global and static; otherwise, we might face weird 
behaviors when different sessions are using/changing different values. Let me 
submit a PR to improve it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16331: [SPARK-18920][HISTORYSERVER]Update outdated date ...

2017-01-07 Thread WangTaoTheTonic
Github user WangTaoTheTonic closed the pull request at:

https://github.com/apache/spark/pull/16331


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16473: [SPARK-19069] [CORE] Expose task 'status' and 'duration'...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16473
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16473: [SPARK-19069] [CORE] Expose task 'status' and 'duration'...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16473
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71028/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16473: [SPARK-19069] [CORE] Expose task 'status' and 'duration'...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71031/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71030/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71029/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16481: [SPARK-19092] [SQL] Save() API of DataFrameWriter should...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16481
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71026/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16481: [SPARK-19092] [SQL] Save() API of DataFrameWriter should...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16481
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16481: [SPARK-19092] [SQL] Save() API of DataFrameWriter should...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16493
  
Also cc @rxin @cloud-fan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16493
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71027/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16493
  
LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16493
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16138
  
**[Test build #71032 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71032/testReport)**
 for PR 16138 at commit 
[`7a13290`](https://github.com/apache/spark/commit/7a13290987d5e36a6d1defb3bb230df276aa).
 * This patch **fails Python style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71032/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16496
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16138
  
**[Test build #71032 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71032/testReport)**
 for PR 16138 at commit 
[`7a13290`](https://github.com/apache/spark/commit/7a13290987d5e36a6d1defb3bb230df276aa).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16496
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71025/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread anabranch
Github user anabranch commented on the issue:

https://github.com/apache/spark/pull/16138
  
I believe now why my previous implementation did not work.

My implementation originally looked like this:

```scala
case class ParseToTimestamp(left: Expression, format: Expression, child: 
Expression)
  extends RuntimeReplaceable {

  def this(left: Expression, format: Expression) = {
  this(left, format, Cast(UnixTimestamp(left, format), TimestampType))
}

  override def checkInputDataTypes(): TypeCheckResult = {
if (left.dataType != StringType) {
  TypeCheckResult.TypeCheckFailure(s"TO_TIMESTAMP requires both inputs 
to be strings")
}
TypeCheckResult.TypeCheckSuccess
  }

  override def flatArguments: Iterator[Any] = Iterator(left, format)
  override def sql: String = s"$prettyName(${left.sql}, ${format.sql})"

  override def prettyName: String = "to_timestamp"
  override def dataType: DataType = TimestampType
}
```

This implementation with a simple example would fail.

```scala
import org.apache.spark.sql.functions._

val ss1 = "2015-07-24 10:00:00"
val ss2 = "2015-07-25 02:02:02"
val df2 = Seq((ss1), (ss2)).toDF("ss")

df2.select(to_timestamp(col("ss"))).show
```
This throws a 

```
org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to 
dataType on unresolved object, tree: 'ss
```

A `Trace` log level shows that the columns are resolved however the error 
originates after analysis and during `CheckInputDataTypes`. This function seeks 
to analyze the `left` input, however because this column is input into a 
`RuntimeReplaceable` function the relevant, and resolved, tree is actually the 
`child` argument - `left` remains unresolved (and therefore throws the above 
error).

I believe this to be the root cause and that has in turn showed me that I 
do not need to perform input validation for this function in the first place. 
Since I only wrap functions, they are performing the exact same input 
validation that I would be. Since no new logic is implemented, there's no point 
in redundantly validating something that will be validated again anyways, 
especially when the system won't let me.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with ...

2017-01-07 Thread anabranch
Github user anabranch commented on a diff in the pull request:

https://github.com/apache/spark/pull/16138#discussion_r95071556
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 ---
@@ -1047,6 +1048,60 @@ case class ToDate(child: Expression) extends 
UnaryExpression with ImplicitCastIn
 }
 
 /**
+ * Parses a column to a date based on the given format.
+ */
+@ExpressionDescription(
+  usage = "_FUNC_(date_str, fmt) - Parses the `left` expression with the 
`fmt` expression.",
+  extended = """
+Examples:
+  > SELECT _FUNC_('2016-12-31', '-MM-dd');
+   2016-12-31
+  """)
+case class ParseToDate(left: Expression, format: Expression, child: 
Expression)
+  extends RuntimeReplaceable {
+
+  def this(left: Expression, format: Expression) = {
+this(left, format, Cast(new ParseToTimestamp(left, format), DateType))
+  }
+
+  def this(left: Expression) = {
+this(left, Literal(""), ToDate(left))
--- End diff --

so this second value gets ignored completely. Not sure exactly how to 
handle this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with ...

2017-01-07 Thread anabranch
Github user anabranch commented on a diff in the pull request:

https://github.com/apache/spark/pull/16138#discussion_r95071450
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
 ---
@@ -389,6 +389,20 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 checkConsistencyBetweenInterpretedAndCodegen(ToDate, DateType)
   }
 
+  test("function parse_to_date") {
+def testDateParse(input: String, format: String, output: String): Unit 
= {
+  val parsed = new ParseToDate(Literal(input), Literal(format))
+  checkEvaluation(parsed, 
DateTimeUtils.fromJavaDate(Date.valueOf((output))), null)
+}
+// This fails but I am not sure why, it doesn't really make sense that 
it would
+// simply because it's the right answer. I don't understand 
CheckEvaluation.
+testDateParse("2015-07-15", "-MM-dd", "2015-07-15")
+  }
+
+  test("function parse_to_timestamp") {
--- End diff --

same here obviously


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with ...

2017-01-07 Thread anabranch
Github user anabranch commented on a diff in the pull request:

https://github.com/apache/spark/pull/16138#discussion_r95071452
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 ---
@@ -342,7 +342,8 @@ object FunctionRegistry {
 expression[CurrentTimestamp]("now"),
 expression[Quarter]("quarter"),
 expression[Second]("second"),
-expression[ToDate]("to_date"),
+expression[ParseToTimestamp]("to_timestamp"),
+expression[ParseToDate]("to_date"),
--- End diff --

I think this is ok to do because I just remapped it, I'm not sure how to 
register multiple functions with the same signature for SQL.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with ...

2017-01-07 Thread anabranch
Github user anabranch commented on a diff in the pull request:

https://github.com/apache/spark/pull/16138#discussion_r95071447
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
 ---
@@ -389,6 +389,20 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 checkConsistencyBetweenInterpretedAndCodegen(ToDate, DateType)
   }
 
+  test("function parse_to_date") {
--- End diff --

I'm confused as to why this is failing. I need some guidance on setting 
these tests up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with ...

2017-01-07 Thread anabranch
Github user anabranch commented on a diff in the pull request:

https://github.com/apache/spark/pull/16138#discussion_r95071721
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -143,6 +143,12 @@ def _():
'measured in radians.',
 }
 
+_functions_2_2 = {
--- End diff --

I want to double check all these docs, but I tried the functions and they 
work!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16487: [SPARK-19107][SQL] support creating hive table wi...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16487#discussion_r95071633
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala 
---
@@ -1289,4 +1290,66 @@ class HiveDDLSuite
   }
 }
   }
+
+  test("create hive serde table with Catalog") {
+withTable("t") {
+  withTempDir { dir =>
+val df = spark.catalog.createExternalTable(
+  "t",
+  "hive",
+  new StructType().add("i", "int"),
+  Map("path" -> dir.getCanonicalPath, "fileFormat" -> "parquet"))
--- End diff --

Maybe, just issue an exception when users do not provide a `path`? 
Otherwise, we have to add new APIs. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16487: [SPARK-19107][SQL] support creating hive table wi...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16487#discussion_r95071626
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -385,6 +380,8 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
 }
 EliminateSubqueryAliases(catalog.lookupRelation(tableIdentWithDB)) 
match {
   // Only do the check if the table is a data source table (the 
relation is a BaseRelation).
+  // TODO(cloud-fan): also check hive table relation here when we 
support overwrite mode
+  // for creating hive tables.
--- End diff --

```Scala
  Seq((1, 2)).toDF("i", 
"j").write.format("parquet").saveAsTable(tableName)
  table(tableName).write.mode(SaveMode.Overwrite).insertInto(tableName)
```

We captured the exceptions when the format is `parquet`. Now, when the 
format is `hive`, should we do the same thing?




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16138
  
**[Test build #71031 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71031/testReport)**
 for PR 16138 at commit 
[`d11063b`](https://github.com/apache/spark/commit/d11063b0b145554204d357ded14ffab84c71ee05).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16138
  
**[Test build #71030 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71030/testReport)**
 for PR 16138 at commit 
[`a2539a6`](https://github.com/apache/spark/commit/a2539a67a0d955deee0e22b6b5edaffcb5d5b67a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16138: [WIP][SPARK-16609] Add to_date/to_timestamp with format ...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16138
  
**[Test build #71029 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71029/testReport)**
 for PR 16138 at commit 
[`8c00745`](https://github.com/apache/spark/commit/8c007457d20dd47ce13e994f14b29522b607ee63).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16473: [SPARK-19069] [CORE] Expose task 'status' and 'duration'...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16473
  
**[Test build #71028 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71028/testReport)**
 for PR 16473 at commit 
[`e4c33ac`](https://github.com/apache/spark/commit/e4c33ac6a3895266df4d48938848044bcbb0e121).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16249: [SPARK-18828][SPARKR] Refactor scripts for R

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16249
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71024/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16249: [SPARK-18828][SPARKR] Refactor scripts for R

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16249
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16249: [SPARK-18828][SPARKR] Refactor scripts for R

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16493
  
**[Test build #71027 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71027/testReport)**
 for PR 16493 at commit 
[`3c779d5`](https://github.com/apache/spark/commit/3c779d59fa54f9ed62a3ebef260b097695c0eff1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16500: [SPARK-19120] [SPARK-19121] Refresh Metadata Cache After...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16500
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71023/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16500: [SPARK-19120] [SPARK-19121] Refresh Metadata Cache After...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16500
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16493
  
test this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16500: [SPARK-19120] [SPARK-19121] Refresh Metadata Cache After...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16400: [SPARK-18941][SQL][DOC] Add a new behavior docume...

2017-01-07 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/16400


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16400: [SPARK-18941][SQL][DOC] Add a new behavior document on `...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16400
  
Thanks! Merging to master/2.1/2.0


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16481: [SPARK-19092] [SQL] Save() API of DataFrameWriter should...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16481
  
**[Test build #71026 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71026/testReport)**
 for PR 16481 at commit 
[`7d3eefb`](https://github.com/apache/spark/commit/7d3eefbc30a14cd8457967170ea705b48c8d05e8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16493: [SPARK-19093][SQL] Cached tables are not used in Subquer...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16493
  
retest this please



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16481: [SPARK-19092] [SQL] Save() API of DataFrameWriter...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16481#discussion_r95070345
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -494,8 +500,13 @@ case class DataSource(
 catalogTable = catalogTable,
 fileIndex = fileIndex)
 sparkSession.sessionState.executePlan(plan).toRdd
-// Replace the schema with that of the DataFrame we just wrote out 
to avoid re-inferring it.
-copy(userSpecifiedSchema = 
Some(data.schema.asNullable)).resolveRelation()
+if (isForWriteOnly) {
+  // Exit earlier and return null
--- End diff --

Sure


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16496
  
**[Test build #71025 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71025/testReport)**
 for PR 16496 at commit 
[`943ebb7`](https://github.com/apache/spark/commit/943ebb7691f5c0591764492747c78e4bbba25b46).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/16496
  
retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16249: [SPARK-18828][SPARKR] Refactor scripts for R

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16249
  
**[Test build #71024 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71024/testReport)**
 for PR 16249 at commit 
[`66fc83c`](https://github.com/apache/spark/commit/66fc83cb349f5cd8a34ed3d272b8e0ab7b0fe423).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16500: [SPARK-19120] [SPARK-19121] Refresh Metadata Cach...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16500#discussion_r95068993
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -392,7 +392,9 @@ case class InsertIntoHiveTable(
 
 // Invalidate the cache.
 sqlContext.sharedState.cacheManager.invalidateCache(table)
-
sqlContext.sessionState.catalog.refreshTable(table.catalogTable.identifier)
+if (partition.nonEmpty) {
+  
sqlContext.sessionState.catalog.refreshTable(table.catalogTable.identifier)
+}
--- End diff --

Actually, we can further limit the calls of `refreshTable`. For example, 
checking whether the format is `parquet` or `orc`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16500: [SPARK-19120] [SPARK-19121] Refresh Metadata Cache After...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16500
  
**[Test build #71023 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71023/testReport)**
 for PR 16500 at commit 
[`0f70e91`](https://github.com/apache/spark/commit/0f70e912402e118a79f48db38c1697baf6905cde).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16500: [SPARK-19120] [SPARK-19121] Refresh Metadata Cache After...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16500
  
cc @ericl @cloud-fan @mallman 

The actual code changes are just two lines. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16312: [SPARK-18862][SPARKR][ML] Split SparkR mllib.R into mult...

2017-01-07 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/16312
  
LGTM. thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16500: [SPARK-19120] [SPARK-19121] Refresh Metadata Cach...

2017-01-07 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

https://github.com/apache/spark/pull/16500

[SPARK-19120] [SPARK-19121] Refresh Metadata Cache After Load Partitioned 
Hive Tables

### What changes were proposed in this pull request?
```Scala
sql("CREATE TABLE tab (a STRING) STORED AS PARQUET")

// This table fetch is to fill the cache with zero leaf files
spark.table("tab").show()

sql(
  s"""
 |LOAD DATA LOCAL INPATH '$newPartitionDir' OVERWRITE
 |INTO TABLE tab
   """.stripMargin)

spark.table("tab").show()
```

In the above example, the returned result is empty after table loading. The 
metadata cache could be out of dated after loading new data into the table, 
because loading/inserting does not update the cache. So far, the metadata cache 
are only used for data source tables. Thus, only `parquet` and `orc` formats 
are facing such issues, because the Hive tables are converted to data source 
tables when 
`spark.sql.hive.convertMetastoreParquet`/`spark.sql.hive.convertMetastoreOrc` 
is on. 

This PR is to refresh the metadata cache after processing the `LOAD DATA` 
command.

In addition, Spark SQL does not convert **partitioned** Hive tables 
(orc/parquet) to data source tables in the write path, but the read path is 
using the metadata cache for both **partitioned** and non-partitioned Hive 
tables (orc/parquet). That means, writing the partitioned parquet/orc tables 
still use `InsertIntoHiveTable`, instead of 
`InsertIntoHadoopFsRelationCommand`. To avoid reading the out-of-dated cache, 
`InsertIntoHiveTable` needs to refresh the metadata cache for partitioned 
tables. Note, it does not need to refresh the cache for non-partitioned 
parquet/orc tables, because it does not call `InsertIntoHiveTable` at all. 

### How was this patch tested?
Added a test case in parquetSuites.scala

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/gatorsmile/spark refreshInsertIntoHiveTable

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/16500.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #16500


commit ea91cb077e4ea307020dc5b3b4ffe9a0d2a4dc88
Author: gatorsmile 
Date:   2017-01-07T18:59:52Z

fix.

commit b7013c2853bf993d82b88c4a605ce921d4593ebe
Author: gatorsmile 
Date:   2017-01-07T22:55:25Z

fix.

commit 27fab56bdac74dac2d7dbd36db4c240d35c89dac
Author: gatorsmile 
Date:   2017-01-08T00:06:47Z

fix.

commit 0f70e912402e118a79f48db38c1697baf6905cde
Author: gatorsmile 
Date:   2017-01-08T00:33:20Z

more test cases.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16312: [SPARK-18862][SPARKR][ML] Split SparkR mllib.R in...

2017-01-07 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/16312#discussion_r95068898
  
--- Diff: R/pkg/R/mllib_utils.R ---
@@ -81,9 +80,11 @@ predict_internal <- function(object, newData) {
 #' model <- read.ml(path)
 #' }
 #' @note read.ml since 2.0.0
-read.ml <- function(path, sparkSession = NULL) {
+read.ml <- function(path) {
   path <- suppressWarnings(normalizePath(path))
-  jobj <- callJStatic("org.apache.spark.ml.r.RWrappers", "load", path, 
sparkSession)
+  sparkSession <- getSparkSession()
+  callJStatic("org.apache.spark.ml.r.RWrappers", "session", sparkSession)
--- End diff --

I'm a bit confused by this - what does this do?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16499: [SPARK-17204][CORE] Fix replicated off heap storage

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16499
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71022/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16499: [SPARK-17204][CORE] Fix replicated off heap storage

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16499
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16499: [SPARK-17204][CORE] Fix replicated off heap storage

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16473: [SPARK-19069] [CORE] Expose task 'status' and 'duration'...

2017-01-07 Thread paragpc
Github user paragpc commented on the issue:

https://github.com/apache/spark/pull/16473
  
```[error]  * method 
this(Long,Int,Int,java.util.Date,java.lang.String,java.lang.String,java.lang.String,Boolean,scala.collection.Seq,scala.Option,scala.Option)Unit
 in class org.apache.spark.status.api.v1.TaskData does not have a correspondent 
in current version
[error]filter with: 
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.this")
[error]  * synthetic method $default$10()scala.Option in object 
org.apache.spark.status.api.v1.TaskData does not have a correspondent in 
current version

[error]filter with: 
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$10")
[error]  * synthetic method $default$11()scala.Option in object 
org.apache.spark.status.api.v1.TaskData does not have a correspondent in 
current version
[error]filter with: 
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.TaskData.$default$11")```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16498: SPARK-9487][Tests] Begin the work of robustifying unit t...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16498
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16498: SPARK-9487][Tests] Begin the work of robustifying unit t...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16498: SPARK-9487][Tests] Begin the work of robustifying unit t...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16498
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71021/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16499: [SPARK-17204][CORE] Fix replicated off heap stora...

2017-01-07 Thread mallman
Github user mallman commented on a diff in the pull request:

https://github.com/apache/spark/pull/16499#discussion_r95066452
  
--- Diff: 
core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala 
---
@@ -375,7 +375,8 @@ class BlockManagerReplicationSuite extends SparkFunSuite
   // Put the block into one of the stores
   val blockId = new TestBlockId(
 "block-with-" + storageLevel.description.replace(" ", 
"-").toLowerCase)
-  stores(0).putSingle(blockId, new Array[Byte](blockSize), 
storageLevel)
+  val testValue = Array.fill[Byte](blockSize)(1)
--- End diff --

Using an array of 1s instead of an array of 0s is my silly, paranoid, OCD 
way of adding a little extra entropy to the test. I think the chance that this 
change in test value will actually affect the outcome of this test is about 0%. 
I will revert to the original test value on request.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16499: [SPARK-17204][CORE] Fix replicated off heap storage

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16499
  
**[Test build #71022 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71022/testReport)**
 for PR 16499 at commit 
[`e49aeca`](https://github.com/apache/spark/commit/e49aeca23ff463fbd9a9cc4db99078c466bfbd56).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16499: [SPARK-17204][CORE] Fix replicated off heap stora...

2017-01-07 Thread mallman
Github user mallman commented on a diff in the pull request:

https://github.com/apache/spark/pull/16499#discussion_r95066296
  
--- Diff: 
core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala 
---
@@ -387,12 +388,23 @@ class BlockManagerReplicationSuite extends 
SparkFunSuite
 testStore => 
blockLocations.contains(testStore.blockManagerId.executorId)
   }.foreach { testStore =>
 val testStoreName = testStore.blockManagerId.executorId
-assert(
-  testStore.getLocalValues(blockId).isDefined, s"$blockId was not 
found in $testStoreName")
-testStore.releaseLock(blockId)
--- End diff --

N.B. We no longer need the `releaseLock` call because we exhaust the 
iterator returned by `getLocalValues`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16499: [SPARK-17204][CORE] Fix replicated off heap stora...

2017-01-07 Thread mallman
GitHub user mallman opened a pull request:

https://github.com/apache/spark/pull/16499

[SPARK-17204][CORE] Fix replicated off heap storage

(Jira: https://issues.apache.org/jira/browse/SPARK-17204)

## What changes were proposed in this pull request?

There are a couple of bugs in the `BlockManager` with respect to support 
for replicated off-heap storage. First, the locally-stored off-heap byte buffer 
is disposed of when it is replicated. It should not be. Second, the replica 
byte buffers are stored as heap byte buffers instead of direct byte buffers 
even when the storage level memory mode is off-heap. This PR addresses both of 
these problems.

## How was this patch tested?

`BlockManagerReplicationSuite` was enhanced to fill in the coverage gaps. 
It now fails if either of the bugs in this PR exist.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/VideoAmp/spark-public 
spark-17204-replicated_off_heap_storage

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/16499.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #16499


commit e49aeca23ff463fbd9a9cc4db99078c466bfbd56
Author: Michael Allman 
Date:   2017-01-02T01:28:12Z

Fix a couple of bugs in replicated off-heap storage

commit 40b6b97ca9013544702433dc5dc388c054daf41a
Author: Michael Allman 
Date:   2017-01-07T21:17:33Z

Shore-up BlockManagerReplicationSuite to identify a couple of bugs with
off-heap storage replication




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #15496: [SPARK-17950] [Python] Match SparseVector behavior with ...

2017-01-07 Thread itg-abby
Github user itg-abby commented on the issue:

https://github.com/apache/spark/pull/15496
  
ping @holdenk , please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16481: [SPARK-19092] [SQL] Save() API of DataFrameWriter...

2017-01-07 Thread jaceklaskowski
Github user jaceklaskowski commented on a diff in the pull request:

https://github.com/apache/spark/pull/16481#discussion_r95065596
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -494,8 +500,13 @@ case class DataSource(
 catalogTable = catalogTable,
 fileIndex = fileIndex)
 sparkSession.sessionState.executePlan(plan).toRdd
-// Replace the schema with that of the DataFrame we just wrote out 
to avoid re-inferring it.
-copy(userSpecifiedSchema = 
Some(data.schema.asNullable)).resolveRelation()
+if (isForWriteOnly) {
+  // Exit earlier and return null
--- End diff --

I'd remove "and return null"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16492: [SPARK-19113][SS][Tests]Set UncaughtExceptionHand...

2017-01-07 Thread jaceklaskowski
Github user jaceklaskowski commented on a diff in the pull request:

https://github.com/apache/spark/pull/16492#discussion_r95065464
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala ---
@@ -235,7 +235,10 @@ trait StreamTest extends QueryTest with 
SharedSQLContext with Timeouts {
*/
   def testStream(
   _stream: Dataset[_],
-  outputMode: OutputMode = OutputMode.Append)(actions: StreamAction*): 
Unit = {
+  outputMode: OutputMode = OutputMode.Append)(actions: StreamAction*): 
Unit = synchronized {
+// `synchronized` is added to prevent the user from calling 
`testStream` concurrently because
--- End diff --

Can you elaborate why `StreamingQueryListener` can be used concurrently? 
I'm reading the comment: "it does not work because something else does not 
work". Why? What's wrong with `StreamingQueryListener`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16492: [SPARK-19113][SS][Tests]Set UncaughtExceptionHand...

2017-01-07 Thread jaceklaskowski
Github user jaceklaskowski commented on a diff in the pull request:

https://github.com/apache/spark/pull/16492#discussion_r95065415
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala ---
@@ -238,7 +238,7 @@ class StreamSuite extends StreamTest {
 }
   }
 
-  testQuietly("fatal errors from a source should be sent to the user") {
+  testQuietly("handle fatal errors thrown from the stream thread 
correctly") {
--- End diff --

`correctly` is superfluous and could be removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16496
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16496
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71019/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16496: [SPARK-16101][SQL] Refactoring CSV write path to be cons...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #13818: [SPARK-15968][SQL] Nonempty partitioned metastore...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/13818#discussion_r95064603
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -298,6 +298,7 @@ case class InsertIntoHiveTable(
 
 // Invalidate the cache.
 sqlContext.sharedState.cacheManager.invalidateCache(table)
+
sqlContext.sessionState.catalog.invalidateTable(table.catalogTable.identifier)
--- End diff --

Let me submit a PR to fix these issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #13818: [SPARK-15968][SQL] Nonempty partitioned metastore...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/13818#discussion_r95064489
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -298,6 +298,7 @@ case class InsertIntoHiveTable(
 
 // Invalidate the cache.
 sqlContext.sharedState.cacheManager.invalidateCache(table)
+
sqlContext.sessionState.catalog.invalidateTable(table.catalogTable.identifier)
--- End diff --

We are facing the same issue in LOAD.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15413: [SPARK-17847][ML] Reduce shuffled data size of Ga...

2017-01-07 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/15413#discussion_r95064061
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala 
---
@@ -126,9 +143,93 @@ class GaussianMixtureSuite extends SparkFunSuite with 
MLlibTestSparkContext
 testEstimatorAndModelReadWrite(gm, dataset,
   GaussianMixtureSuite.allParamSettings, checkModelData)
   }
+
+  test("univariate dense/sparse data with two clusters") {
+val weights = Array(2.0 / 3.0, 1.0 / 3.0)
+val means = Array(Vectors.dense(5.1604), Vectors.dense(-4.3673))
+val covs = Array(Matrices.dense(1, 1, Array(0.86644)), 
Matrices.dense(1, 1, Array(1.1098)))
+val gaussians = means.zip(covs).map { case (mean, cov) =>
+  new MultivariateGaussian(mean, cov)
+}
+val expected = new GaussianMixtureModel("dummy", weights, gaussians)
+
+Seq(denseDataset, sparseDataset).foreach { dataset =>
+  val actual = new GaussianMixture().setK(2).setSeed(seed).fit(dataset)
+  modelEquals(expected, actual)
+}
+  }
+
+  test("check distributed decomposition") {
--- End diff --

This test only checks that when we distribute the computation that it 
produces a model, i.e. that it doesn't fail. So, AFAICT we don't have any test 
right now that checks that when we distribute the computation it produces a 
_correct_ model. I think it's a good idea to have that here. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15413: [SPARK-17847][ML] Reduce shuffled data size of Ga...

2017-01-07 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/15413#discussion_r95064361
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala 
---
@@ -141,4 +242,37 @@ object GaussianMixtureSuite {
 "maxIter" -> 2,
 "tol" -> 0.01
   )
+
+  val denseData = Seq(
+Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220),
+Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118),
+Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322),
+Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026),
+Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734)
+  )
+
+  val decompositionData: Seq[Vector] = Seq.tabulate(25) { i: Int =>
+Vectors.dense(Array.tabulate(50)(i + _.toDouble))
+  }
+
+  val rData = Seq(
+Vectors.dense(-0.6264538, 0.1836433), Vectors.dense(-0.8356286, 
1.5952808),
+Vectors.dense(0.3295078, -0.8204684), Vectors.dense(0.4874291, 
0.7383247),
+Vectors.dense(0.5757814, -0.3053884), Vectors.dense(1.5117812, 
0.3898432),
+Vectors.dense(-0.6212406, -2.2146999), Vectors.dense(11.1249309, 
9.9550664),
+Vectors.dense(9.9838097, 10.9438362), Vectors.dense(10.8212212, 
10.5939013),
+Vectors.dense(10.9189774, 10.7821363), Vectors.dense(10.0745650, 
8.0106483),
+Vectors.dense(10.6198257, 9.9438713), Vectors.dense(9.8442045, 
8.5292476),
+Vectors.dense(9.5218499, 10.4179416)
+  )
+
+  case class FeatureData(features: Vector)
+
+  def modelEquals(m1: GaussianMixtureModel, m2: GaussianMixtureModel): 
Unit = {
+assert(m1.weights.length === m2.weights.length)
+for (i <- m1.weights.indices) {
+  assert(m1.gaussians(i).mean ~== m2.gaussians(i).mean absTol 1E-3)
--- End diff --

Why not also check the weights here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #15413: [SPARK-17847][ML] Reduce shuffled data size of Ga...

2017-01-07 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/15413#discussion_r95064130
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala 
---
@@ -126,9 +143,93 @@ class GaussianMixtureSuite extends SparkFunSuite with 
MLlibTestSparkContext
 testEstimatorAndModelReadWrite(gm, dataset,
   GaussianMixtureSuite.allParamSettings, checkModelData)
   }
+
--- End diff --

In most of the other test suites in ML we have a test that checks the 
prediction/transform methods. For example, checking that the prediction always 
matches the highest probability, checking that probabilities sum to one. I 
don't see much reason to diverge from that pattern here, what do you think 
@yanboliang?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16498: SPARK-9487][Tests] Begin the work of robustifying unit t...

2017-01-07 Thread skanjila
Github user skanjila commented on the issue:

https://github.com/apache/spark/pull/16498
  
Whats different is that this time around I am only making one change at a 
time (in this case ContextCleanerSuite), fixing the unit tests by backtracking 
from the Jenkins failures.  In this strategy I can focus my efforts on 
rewriting unit tests around one module or class as opposed to dealing with 3 or 
4 failures at a time. If you dont feel like this approach is worth pursuing you 
can close this JIRA but I at least thought I would purse this and see how far I 
get.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #13818: [SPARK-15968][SQL] Nonempty partitioned metastore...

2017-01-07 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/13818#discussion_r95064132
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -298,6 +298,7 @@ case class InsertIntoHiveTable(
 
 // Invalidate the cache.
 sqlContext.sharedState.cacheManager.invalidateCache(table)
+
sqlContext.sessionState.catalog.invalidateTable(table.catalogTable.identifier)
--- End diff --

We only need to invalidate the cache for partitioned tables. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #16490: [SPARK-19106][DOCS] Styling for the configuration docs i...

2017-01-07 Thread srowen
Github user srowen commented on the issue:

https://github.com/apache/spark/pull/16490
  
Merged to master/2.1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #15770: [SPARK-15784][ML]:Add Power Iteration Clustering to spar...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15770
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/71020/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #15770: [SPARK-15784][ML]:Add Power Iteration Clustering to spar...

2017-01-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15770
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark issue #15770: [SPARK-15784][ML]:Add Power Iteration Clustering to spar...

2017-01-07 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16490: [SPARK-19106][DOCS] Styling for the configuration...

2017-01-07 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/16490


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16491: [SPARK-19110][ML][MLLIB]:DistributedLDAModel retu...

2017-01-07 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/16491


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-07 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95061617
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
 ---
@@ -465,6 +466,48 @@ class SessionCatalogSuite extends SparkFunSuite {
 assert(plan == SubqueryAlias("range", tmpView, 
Option(TableIdentifier("vw1"
   }
 
+  test("look up view relation") {
+val externalCatalog = newBasicCatalog()
+val sessionCatalog = new SessionCatalog(externalCatalog)
+val metadata = externalCatalog.getTable("db3", "view1")
+sessionCatalog.setCurrentDatabase("default")
+// Look up a view.
+assert(metadata.viewText.isDefined)
+val view = normalizeExprIds(View(desc = metadata, output = 
metadata.schema.toAttributes,
+  child = CatalystSqlParser.parsePlan(metadata.viewText.get)))
+assert(
+  
normalizeExprIds(sessionCatalog.lookupRelation(TableIdentifier("view1", 
Some("db3"
+== SubqueryAlias("view1", view, Some(TableIdentifier("view1", 
Some("db3")
+// Look up a view using current database of the session catalog.
+sessionCatalog.setCurrentDatabase("db3")
+assert(
+  
normalizeExprIds(sessionCatalog.lookupRelation(TableIdentifier("view1")))
+== SubqueryAlias("view1", view, Some(TableIdentifier("view1"
+  }
+
+  /**
--- End diff --

Just make `SessionCatalogSuite` a `PlanTest`, instead of copying code from 
PlanTest...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-07 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95061594
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
 ---
@@ -860,6 +864,25 @@ abstract class CatalogTestUtils {
   bucketSpec = Some(BucketSpec(4, Seq("col1"), Nil)))
   }
 
+  def newView(
+  name: String,
+  database: Option[String] = None): CatalogTable = {
+CatalogTable(
+  identifier = TableIdentifier(name, database),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType()
+.add("col1", "int")
+.add("col2", "string")
+.add("a", "int")
+.add("b", "string"),
+  viewOriginalText = Some("SELECT * FROM tbl1"),
+  viewText = Some("SELECT `gen_attr_0` AS `col1`, `gen_attr_1` AS 
`col2`, `gen_attr_2` AS " +
--- End diff --

why not `SELECT * FROM tbl1`? That should work now :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-07 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95061253
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -767,7 +857,45 @@ class Analyzer(
 }
   }
 
- /**
+  /**
+   * Alias the output of a view's child to the output of the view, the 
corresponding attribute is
+   * searched by name. If the corresponding attribute is not found, throw 
an AnalysisException.
+   * On the resolution of the view, the output attributes are generated 
from the view schema, and
+   * the view query is resolved later. After the view attributes have been 
stabilized(when the
+   * resolution batch has finished), we add a Project operator over the 
child, so that we could
+   * alias the output of the child plan to the view's output attributes.
+   * TODO: Also check the dataTypes and nullabilites of the output.
--- End diff --

We really should throw an exception if the datatypes do not match up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



  1   2   3   >