[GitHub] spark pull request: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118745945
  
Thanks for looking at this. Going to merge 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: [SPARK-7190] [SPARK-8804] [SPARK-7815] unsafe ...

2015-07-05 Thread tarekauel
Github user tarekauel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7197#discussion_r33909453
  
--- Diff: 
unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java ---
@@ -44,60 +47,77 @@
 5, 5, 5, 5,
 6, 6, 6, 6};
 
+  /**
+   * Note: `bytes` will be hold by returned UTF8String.
+   */
   public static UTF8String fromBytes(byte[] bytes) {
-return (bytes != null) ? new UTF8String().set(bytes) : null;
+if (bytes != null) {
+  return new UTF8String(bytes, BYTE_ARRAY_OFFSET, bytes.length);
+} else {
+  return null;
+}
   }
 
   public static UTF8String fromString(String str) {
-return (str != null) ? new UTF8String().set(str) : null;
-  }
-
-  /**
-   * Updates the UTF8String with String.
-   */
-  protected UTF8String set(final String str) {
+if (str == null) return null;
 try {
-  bytes = str.getBytes("utf-8");
+  return fromBytes(str.getBytes("utf-8"));
 } catch (UnsupportedEncodingException e) {
   // Turn the exception into unchecked so we can find out about it at 
runtime, but
   // don't need to add lots of boilerplate code everywhere.
-  PlatformDependent.throwException(e);
+  throwException(e);
+  return null;
 }
-return this;
   }
 
-  /**
-   * Updates the UTF8String with byte[], which should be encoded in UTF-8.
-   */
-  protected UTF8String set(final byte[] bytes) {
-this.bytes = bytes;
-return this;
+  public UTF8String(Object base, long offset, int size) {
+this.base = base;
+this.offset = offset;
+this.numBytes = size;
   }
 
   /**
* Returns the number of bytes for a code point with the first byte as 
`b`
* @param b The first byte of a code point
*/
-  public int numBytes(final byte b) {
+  public int numBytesForFirstByte(final byte b) {
--- End diff --

This could method could be static


---
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: [SPARK-8270][SQL] levenshtein distance

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7214#discussion_r33909072
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1580,22 +1580,37 @@ object functions {
   
//
 
   /**
-   * Computes the length of a given string value
-   *
+   * Computes the length of a given string value.
+   * 
--- End diff --

add it to utf8string, and just call it from 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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118744074
  
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 pull request: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118744073
  
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 pull request: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118743990
  
  [Test build #36558 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36558/console)
 for   PR 7232 at commit 
[`934d3d1`](https://github.com/apache/spark/commit/934d3d1b87e3bdd7f93bbbf5f3e7bb02642fde82).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class Decode(bin: Expression, charset: Expression)`



---
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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118743989
  
  [Test build #36557 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36557/console)
 for   PR 7231 at commit 
[`8be4723`](https://github.com/apache/spark/commit/8be4723f3f8e1749a4274a69a103140e472328f0).
 * 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: [SPARK-8753][SQL][WIP] Create an IntervalType ...

2015-07-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7226#issuecomment-118743957
  
We should add a way to cast interval into string and string to interval 
too. That can go in a separate pull request though.



---
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: [SPARK-8753][SQL][WIP] Create an IntervalType ...

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7226#discussion_r33908974
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -1458,4 +1458,14 @@ class SQLQuerySuite extends QueryTest with 
BeforeAndAfterAll with SQLTestUtils {
   checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), 
Row(1, 2)))
 }
   }
+
+  test("SPARK-8753: add interval type") {
+val df = sql("select interval 3 year, interval -14 month, interval 99 
second, interval -4 day")
+checkAnswer(df, Row(36, -14, 99L, -345600L))
+withTempPath(f => {
--- End diff --

add an inline comment here saying we don't yet support saving out interval 
data types.



---
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: [SPARK-8753][SQL][WIP] Create an IntervalType ...

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7226#discussion_r33908958
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala ---
@@ -303,6 +303,10 @@ private[sql] object ResolvedDataSource {
   mode: SaveMode,
   options: Map[String, String],
   data: DataFrame): ResolvedDataSource = {
+if (data.schema.map(_.dataType).exists(
+  t => t == YearMonthIntervalType || t == DayTimeIntervalType)) {
--- End diff --

once you create an IntervalType adt, use 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: [SPARK-8759][SQL] add default eval to binary a...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7157#issuecomment-118743737
  
  [Test build #36562 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36562/consoleFull)
 for   PR 7157 at commit 
[`2876cbd`](https://github.com/apache/spark/commit/2876cbd0e8398afef3d78346b2e282befed83832).


---
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: [SPARK-8401] [Build] Scala version switching b...

2015-07-05 Thread mallman
Github user mallman commented on the pull request:

https://github.com/apache/spark/pull/6832#issuecomment-118743436
  
@srowen I just returned from my vacation abroad and am catching up. Sorry 
for the wait. I'll take a look at this tomorrow. Cheers.


---
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: [SPARK-8759][SQL] add default eval to binary a...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7157#issuecomment-118743342
  
Build started.


---
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: [SPARK-8759][SQL] add default eval to binary a...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7157#issuecomment-118743331
  
 Build triggered.


---
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: [SPARK-8270][SQL] levenshtein distance

2015-07-05 Thread tarekauel
Github user tarekauel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7214#discussion_r33908578
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1580,22 +1580,37 @@ object functions {
   
//
 
   /**
-   * Computes the length of a given string value
-   *
+   * Computes the length of a given string value.
+   * 
--- End diff --

@rxin Can I implement this as part of `UTF8String` or should it be in 
stringOperations? 


---
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: [SPARK-8018][MLlib]KMeans should accept initia...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/6737#issuecomment-118738599
  
  [Test build #36561 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36561/console)
 for   PR 6737 at commit 
[`06d13ef`](https://github.com/apache/spark/commit/06d13ef3bd78751cc4ef8996f3a5f79ebcddd099).
 * 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: [SPARK-8018][MLlib]KMeans should accept initia...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/6737#issuecomment-118738773
  
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 pull request: [SPARK-8811][SQL] Read array struct data from ...

2015-07-05 Thread liancheng
Github user liancheng commented on the pull request:

https://github.com/apache/spark/pull/7209#issuecomment-118737760
  
@scwf Yeah, I didn't make the `element` to `array` change in #7231. It 
would be good to have one, either based on this PR or open a new one. The 
tricky part is it needs parquet-avro for writing test case. We may generate a 
Parquet file with parquet-avro and then add it as a resource.


---
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: [SPARK-8753][SQL][WIP] Create an IntervalType ...

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7226#discussion_r33908086
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala
 ---
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.types
+
+import scala.math.Ordering
+import scala.reflect.runtime.universe.typeTag
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.catalyst.ScalaReflectionLock
+
+@DeveloperApi
+class DayTimeIntervalType private() extends AtomicType {
+  // The companion object and this class is separated so the companion 
object also subclasses
+  // this type. Otherwise, the companion object would be of type 
"THIS_TYPE$" in byte code.
+  // Defined with a private constructor so the companion object is the 
only possible instantiation.
+  private[sql] type InternalType = Long
+
+  @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized 
{ typeTag[InternalType] }
+
+  private[sql] val ordering = implicitly[Ordering[InternalType]]
+
+  /**
+   * The default size of a value of the DayTimeIntervalType is 8 bytes.
--- End diff --

maybe remove this comment. not very useful to have it 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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread yijieshen
Github user yijieshen commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118736827
  
Tried this out locally and works fine. LGTM as well.


---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118735867
  
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 pull request: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118735791
  
  [Test build #36560 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36560/console)
 for   PR 7084 at commit 
[`24728e4`](https://github.com/apache/spark/commit/24728e4c35b422457c82ddc540c7de5ff8c47fd8).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class CountVectorizerModel (override val uid: String, val vocabulary: 
Array[String])`
  * `case class ScalaUDF(`
  * `case class CurrentDate() extends LeafExpression `
  * `case class CurrentTimestamp() extends LeafExpression `
  * `case class Hex(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class UnHex(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class ShiftLeft(left: Expression, right: Expression)`
  * `case class ShiftRight(left: Expression, right: Expression)`
  * `case class ShiftRightUnsigned(left: Expression, right: Expression)`
  * `case class Levenshtein(left: Expression, right: Expression) extends 
BinaryExpression`
  * `case class Ascii(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Base64(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class UnBase64(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Decode(bin: Expression, charset: Expression) extends 
Expression with ExpectsInputTypes `
  * `case class Encode(value: Expression, charset: Expression)`
  * `case class UserDefinedFunction protected[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: [SPARK-8271][SQL]string function: soundex

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7115#discussion_r33907699
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -301,6 +303,27 @@ case class StringLength(child: Expression) extends 
UnaryExpression with ExpectsI
 }
 
 /**
+ * A function that return soundex code of the given string expression.
+ */
+case class SoundEx(child: Expression) extends UnaryExpression with 
ExpectsInputTypes {
+  @transient lazy val so = new Soundex
+
+  override def dataType: DataType = StringType
+
+  override def inputTypes: Seq[DataType] = Seq(StringType)
+
+  override def eval(input: InternalRow): Any = {
+val string = child.eval(input)
+if (string == null) {
+  null
+} else {
+  val str: String = string.toString
+  UTF8String.fromString(so.encode(str))
--- End diff --

also can you copy the body of common codec's soundex function into 
UTF8String?



---
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: [SPARK-8271][SQL]string function: soundex

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7115#discussion_r33907684
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -301,6 +303,27 @@ case class StringLength(child: Expression) extends 
UnaryExpression with ExpectsI
 }
 
 /**
+ * A function that return soundex code of the given string expression.
+ */
+case class SoundEx(child: Expression) extends UnaryExpression with 
ExpectsInputTypes {
+  @transient lazy val so = new Soundex
+
+  override def dataType: DataType = StringType
+
+  override def inputTypes: Seq[DataType] = Seq(StringType)
+
+  override def eval(input: InternalRow): Any = {
+val string = child.eval(input)
+if (string == null) {
+  null
+} else {
+  val str: String = string.toString
+  UTF8String.fromString(so.encode(str))
--- End diff --

you need to return null if there is an exception


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread cloud-fan
Github user cloud-fan commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118735055
  
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 pull request: [SPARK-8271][SQL]string function: soundex

2015-07-05 Thread HuJiayin
Github user HuJiayin commented on the pull request:

https://github.com/apache/spark/pull/7115#issuecomment-118735097
  
@rxin @davies @liancheng can you trigger the unit test?


---
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: [SPARK-8018][MLlib]KMeans should accept initia...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/6737#issuecomment-118733142
  
  [Test build #36561 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36561/consoleFull)
 for   PR 6737 at commit 
[`06d13ef`](https://github.com/apache/spark/commit/06d13ef3bd78751cc4ef8996f3a5f79ebcddd099).


---
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: [SPARK-8018][MLlib]KMeans should accept initia...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/6737#issuecomment-118732111
  
 Merged build triggered.


---
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: [SPARK-8018][MLlib]KMeans should accept initia...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/6737#issuecomment-118732126
  
Merged build started.


---
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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7219#issuecomment-118730797
  
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 pull request: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7219#issuecomment-118730681
  
  [Test build #36556 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36556/console)
 for   PR 7219 at commit 
[`537579a`](https://github.com/apache/spark/commit/537579ad94036cb392d82d3802eccc38d72fb861).
 * 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: [SPARK-4176][WIP] Support decimal types with p...

2015-07-05 Thread liancheng
Github user liancheng commented on the pull request:

https://github.com/apache/spark/pull/6796#issuecomment-118730215
  
Hey @rtreffer, just want to make sure whether you are still working on 
this? I'm asking because I just opened #7231 to refactor Parquet read path for 
interoperability and backwards-compatibility, which also touches the decimal 
parts. I believe the new [`CatalystDecimalConverter`] [1] already covers the 
read path of decimals with precision > 18, which means this PR can be further 
simplified. Just in case you don't have time to continue this PR, I'm happy to 
fork your branch and get it merged (will still list you as the main author).

[1]: 
https://github.com/apache/spark/pull/7231/files#diff-1d6c363c04155a9328fe1f5bd08a2f90R237


---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118729997
  
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 pull request: Fixed First and Last aggregates to compute fir...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7233#issuecomment-118727714
  
Can one of the admins verify this patch?


---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118726940
  
  [Test build #36560 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36560/consoleFull)
 for   PR 7084 at commit 
[`24728e4`](https://github.com/apache/spark/commit/24728e4c35b422457c82ddc540c7de5ff8c47fd8).


---
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: Fixed First and Last aggregates to compute fir...

2015-07-05 Thread ggupta81
GitHub user ggupta81 opened a pull request:

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

Fixed First and Last aggregates to compute first and last value per G…

In current implementation, First and Last aggregates were calculating the 
values for entire DataFrame partition and then the same value was returned for 
all GroupedData in the partition.
Fixed the First and Last aggregates to compute first and last value per 
GroupedData instead of entire DataFrame

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

$ git pull https://github.com/ggupta81/spark branch-1.4

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

https://github.com/apache/spark/pull/7233.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 #7233


commit 11db44bddd6d1eb3fe56cb7e9d852ae989aa123d
Author: Gaurav Gupta 
Date:   2015-07-06T05:24:46Z

Fixed First and Last aggregates to compute first and last value per 
GroupedData instead of entire DataFrame




---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118725388
  
Merged build started.


---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118725374
  
 Merged build triggered.


---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread hhbyyh
Github user hhbyyh commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118725060
  
Thank a lot @jkbradley. I sent an update with:
1. change the class name to CountVectorizerModel.
2. make vocab a val.
3. change minTermCount to minTermFreq and improve doc.
4. other minor fix.



---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118724246
  
Merged build started.


---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7084#issuecomment-118724201
  
 Merged build triggered.


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread yijieshen
Github user yijieshen commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118721775
  
Yep, just what I need :)


---
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: [SPARK-8753][SQL][WIP] Create an IntervalType ...

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7226#discussion_r33906404
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala ---
@@ -58,6 +58,7 @@ class SqlParser extends AbstractSparkSQLParser with 
DataTypeParser {
   protected val BY = Keyword("BY")
   protected val CASE = Keyword("CASE")
   protected val CAST = Keyword("CAST")
+  protected val DAY = Keyword("DAY")
--- End diff --

can you add a unit test to make sure we can still use "day" as a column 
name? 

if not, I think we need to make sure the identifier can include day/hour etc


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118721345
  
  [Test build #36558 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36558/consoleFull)
 for   PR 7232 at commit 
[`934d3d1`](https://github.com/apache/spark/commit/934d3d1b87e3bdd7f93bbbf5f3e7bb02642fde82).


---
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: fix percentPruned log message

2015-07-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7227#issuecomment-118720578
  
cc @liancheng 


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118720476
  
 Merged build triggered.


---
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: fix percentPruned log message

2015-07-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7227#issuecomment-118720548
  
Jenkins, test this please.

(Did something go wrong with Jenkins?)


---
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: [SPARK-7050][build] Keep kafka-assembly maven ...

2015-07-05 Thread jerryshao
Github user jerryshao commented on the pull request:

https://github.com/apache/spark/pull/5632#issuecomment-118720543
  
Hi @srowen , if the assembly jar is not found using this pattern 
`target/scala-*/spark-streaming-kafka-assembly-*.jar`, the python Kafka test 
will not be run at all.


---
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: [SPARK-8753][SQL][WIP] Create an IntervalType ...

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7226#discussion_r33906339
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala
 ---
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.types
+
+import scala.math.Ordering
+import scala.reflect.runtime.universe.typeTag
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.catalyst.ScalaReflectionLock
+
+@DeveloperApi
+class DayTimeIntervalType private() extends AtomicType {
--- End diff --

let's create an abstract data type for IntervalType.


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118720493
  
Merged build started.


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7232#issuecomment-118719989
  
cc @yijieshen I think you were looking for this in round?


---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread rxin
GitHub user rxin opened a pull request:

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

[SPARK-8831][SQL] Support AbstractDataType in TypeCollection.

Otherwise it is impossible to declare an expression supporting DecimalType.


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

$ git pull https://github.com/rxin/spark typecollection-adt

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

https://github.com/apache/spark/pull/7232.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 #7232


commit 934d3d1b87e3bdd7f93bbbf5f3e7bb02642fde82
Author: Reynold Xin 
Date:   2015-07-06T05:04:09Z

[SPARK-8831][SQL] Support AbstractDataType in TypeCollection.




---
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: [SPARK-8831][SQL] Support AbstractDataType in ...

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7232#discussion_r33906266
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
 ---
@@ -708,8 +708,6 @@ object HiveTypeCoercion {
 case (NullType, target) => Cast(e, target.defaultConcreteType)
 
 // Implicit cast among numeric types
-// If input is decimal, and we expect a decimal type, just use the 
input.
-case (_: DecimalType, DecimalType) => e
--- End diff --

cc @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 pull request: [SPARK-8233][SQL] misc function: hash

2015-07-05 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/6971#discussion_r33906234
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
 ---
@@ -47,4 +48,44 @@ case class Md5(child: Expression)
   "org.apache.spark.unsafe.types.UTF8String.fromString" +
 s"(org.apache.commons.codec.digest.DigestUtils.md5Hex($c))")
   }
+
+}
+
+/**
+ * A function that returns a hash value of the argument
+ */
+case class Hash(children: Expression*) extends Expression {
+
+  override def foldable: Boolean = children.forall(_.foldable)
+  override def nullable: Boolean = children(0).nullable
+  override def dataType: DataType = IntegerType
+
+  override def eval(input: InternalRow): Any = {
+val arglist = children.map(_.eval(input).asInstanceOf[AnyRef])
+hashCode(arglist)
+  }
+
+  def hashCode(v: Any): Int = v match {
+case null => 0
+case arr: Array[_] => {
+  arr.foldLeft(0) { (acc, n) => acc + hashCode(n) }
+}
+case seq: Seq[_] => {
+  seq.foldLeft(0) { (acc, n) => acc + hashCode(n) }
+}
+case m: Map[_, _] => {
+  var res: Int = 0
+  m.foreach { case (k, v) => res += (hashCode(k) + hashCode(v)) }
+  res
+}
+case r: Row => {
+  var res: Int = 0
+  for (i <- 0 until r.length) {
+res += 31 * res + hashCode(r.get(i))
+  }
+  res
+}
+case others => others.hashCode()
+  }
+
--- End diff --

Actually I am thinking if we need to add wrappers for ALL of the internal 
data type, which hide the details for the expression developers. e.g the 
hashCode, serialization / deseriazliation, equality etc. Currently seems the 
expression developers needs to keep eyes on those details.

Any ideas?


---
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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/7219#issuecomment-118719759
  
I made a minor comment regarding the name. Pending that small update this 
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 pull request: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/7219#discussion_r33906211
  
--- Diff: pom.xml ---
@@ -1826,6 +1830,26 @@
   
 
 
+
+  
+  release-profile
--- End diff --

Actually maybe `release-publish` is the best, since this is really specific 
to publishing the release to maven.


---
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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/7219#discussion_r33906199
  
--- Diff: pom.xml ---
@@ -1826,6 +1830,26 @@
   
 
 
+
+  
+  release-profile
--- End diff --

Can you rename this to just be called `release`? All of the profiles are, 
well, profiles. As a convention we don't include `profile` in the name.


---
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: [SPARK-8233][SQL] misc function: hash

2015-07-05 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/6971#discussion_r33906097
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
 ---
@@ -47,4 +48,44 @@ case class Md5(child: Expression)
   "org.apache.spark.unsafe.types.UTF8String.fromString" +
 s"(org.apache.commons.codec.digest.DigestUtils.md5Hex($c))")
   }
+
+}
+
+/**
+ * A function that returns a hash value of the argument
+ */
+case class Hash(children: Expression*) extends Expression {
+
+  override def foldable: Boolean = children.forall(_.foldable)
+  override def nullable: Boolean = children(0).nullable
+  override def dataType: DataType = IntegerType
+
+  override def eval(input: InternalRow): Any = {
+val arglist = children.map(_.eval(input).asInstanceOf[AnyRef])
+hashCode(arglist)
+  }
+
+  def hashCode(v: Any): Int = v match {
+case null => 0
+case arr: Array[_] => {
+  arr.foldLeft(0) { (acc, n) => acc + hashCode(n) }
+}
+case seq: Seq[_] => {
+  seq.foldLeft(0) { (acc, n) => acc + hashCode(n) }
+}
+case m: Map[_, _] => {
+  var res: Int = 0
+  m.foreach { case (k, v) => res += (hashCode(k) + hashCode(v)) }
+  res
+}
+case r: Row => {
+  var res: Int = 0
+  for (i <- 0 until r.length) {
+res += 31 * res + hashCode(r.get(i))
+  }
+  res
+}
+case others => others.hashCode()
+  }
+
--- End diff --

That's required a recursive revisting all of its elements for the complex 
data type, code gen here probably be difficult (not easy to maintain compare to 
the performance gain).


---
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: [SPARK-8270][SQL] levenshtein distance

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7214#discussion_r33906065
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1580,22 +1580,37 @@ object functions {
   
//
 
   /**
-   * Computes the length of a given string value
-   *
+   * Computes the length of a given string value.
+   * 
--- End diff --

Note I filed https://issues.apache.org/jira/browse/SPARK-8830


---
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: [SQL][Minor] Update the DataFrame API for enco...

2015-07-05 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-8221][SQL]Add pmod function

2015-07-05 Thread zhichao-li
Github user zhichao-li commented on the pull request:

https://github.com/apache/spark/pull/6783#issuecomment-118715210
  
cc @rxin @davies  could you take a look at 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: [SQL][Minor] Update the DataFrame API for enco...

2015-07-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7230#issuecomment-118714652
  
Thanks - merging 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: [SPARK-8549][SparkR] Fix the line length of Sp...

2015-07-05 Thread yu-iskw
Github user yu-iskw commented on the pull request:

https://github.com/apache/spark/pull/7204#issuecomment-118711831
  
@rxin Thank you for letting me know. I have updated my github email in my 
account profile.


---
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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118710803
  
  [Test build #36557 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36557/consoleFull)
 for   PR 7231 at commit 
[`8be4723`](https://github.com/apache/spark/commit/8be4723f3f8e1749a4274a69a103140e472328f0).


---
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: [SPARK-8233][SQL] misc function: hash

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/6971#discussion_r33905466
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
 ---
@@ -47,4 +48,44 @@ case class Md5(child: Expression)
   "org.apache.spark.unsafe.types.UTF8String.fromString" +
 s"(org.apache.commons.codec.digest.DigestUtils.md5Hex($c))")
   }
+
+}
+
+/**
+ * A function that returns a hash value of the argument
+ */
+case class Hash(children: Expression*) extends Expression {
+
+  override def foldable: Boolean = children.forall(_.foldable)
+  override def nullable: Boolean = children(0).nullable
+  override def dataType: DataType = IntegerType
+
+  override def eval(input: InternalRow): Any = {
+val arglist = children.map(_.eval(input).asInstanceOf[AnyRef])
+hashCode(arglist)
+  }
+
+  def hashCode(v: Any): Int = v match {
+case null => 0
+case arr: Array[_] => {
+  arr.foldLeft(0) { (acc, n) => acc + hashCode(n) }
+}
+case seq: Seq[_] => {
+  seq.foldLeft(0) { (acc, n) => acc + hashCode(n) }
+}
+case m: Map[_, _] => {
+  var res: Int = 0
+  m.foreach { case (k, v) => res += (hashCode(k) + hashCode(v)) }
+  res
+}
+case r: Row => {
+  var res: Int = 0
+  for (i <- 0 until r.length) {
+res += 31 * res + hashCode(r.get(i))
+  }
+  res
+}
+case others => others.hashCode()
+  }
+
--- End diff --

+1 on codegen support. much fewer branches.



---
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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118710134
  
 Merged build triggered.


---
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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118710172
  
Merged build started.


---
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: [SPARK-8270][SQL] levenshtein distance

2015-07-05 Thread tarekauel
Github user tarekauel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7214#discussion_r33905262
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1580,22 +1580,37 @@ object functions {
   
//
 
   /**
-   * Computes the length of a given string value
-   *
+   * Computes the length of a given string value.
+   * 
--- End diff --

Ok, got 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: [SPARK-8549][SparkR] Fix the line length of Sp...

2015-07-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7204#issuecomment-118709270
  
@yu-iskw you might want to update your github email to include the email 
address you used for your commits. Otherwise it doesn't show up in the github 
metadata.



---
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: [SPARK-8270][SQL] levenshtein distance

2015-07-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7214#discussion_r33905094
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1580,22 +1580,37 @@ object functions {
   
//
 
   /**
-   * Computes the length of a given string value
-   *
+   * Computes the length of a given string value.
+   * 
--- End diff --

A few benefits:

1. We don't need to depend on commons-util. In general it is good to reduce 
external dependency due to dependency conflicts.

2. Our version ideally should work directly against UTF8String, so we don't 
need conversion.



---
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: [SPARK-8270][SQL] levenshtein distance

2015-07-05 Thread tarekauel
Github user tarekauel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7214#discussion_r33904993
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
@@ -1580,22 +1580,37 @@ object functions {
   
//
 
   /**
-   * Computes the length of a given string value
-   *
+   * Computes the length of a given string value.
+   * 
--- End diff --

Sure, but is there a benefit? The `StringUtils` implementation does look 
reasonable, doesn't it? Do you have a special concern about their 
implementation? 
(http://grepcode.com/file/repo1.maven.org/maven2/org.apache.commons/commons-lang3/3.0/org/apache/commons/lang3/StringUtils.java#StringUtils.getLevenshteinDistance%28java.lang.CharSequence%2Cjava.lang.CharSequence%29)
 


---
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: [SPARK-8269][SQL]string function: initcap

2015-07-05 Thread tarekauel
Github user tarekauel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7208#discussion_r33904801
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -298,3 +299,56 @@ case class StringLength(child: Expression) extends 
UnaryExpression with ExpectsI
 
   override def prettyName: String = "length"
 }
+
+/**
+ *  A function that return soundex code of the given string expression.
+ */
+case class SoundEx(child: Expression) extends UnaryExpression with 
ExpectsInputTypes {
+  @transient lazy val so = new Soundex
+
+  override def dataType: DataType = StringType
+
+  override def inputTypes: Seq[DataType] = Seq(StringType)
+
+  override def eval(input: InternalRow): Any = {
+val string = child.eval(input)
+if (string == null) {
+  null
+} else {
+  val str: String = string.toString
--- End diff --

`@transient lazy val so = new Soundex` (Line 307) wouldn't be evaluated. 
The object wouldn't be created. I don't know if it's worth to take care of it. 
I just wanted to mention it, because I noticed 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: [SPARK-5259][CORE]Make sure shuffle metadata a...

2015-07-05 Thread suyanNone
Github user suyanNone commented on the pull request:

https://github.com/apache/spark/pull/4055#issuecomment-118708012
  
@squito oh...I had skipped it...
1) Task attempt now is described in `TaskInfo` in Spark `TaskSetManager`. 
`TaskSetManager` is responsible for completing task attempt `TaskInfo`, which 
identified by `TaskID`.  and if `TaskSetManager` is think a attempt was 
succeed, then call `dagScheduler` to complete the `tasks(index)` which is a 
`Task`.
so it will make some sense that identified a `Task` by stageId and 
partitionID?










 


---
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: [SPARK-8549][SparkR] Fix the line length of Sp...

2015-07-05 Thread yu-iskw
Github user yu-iskw commented on the pull request:

https://github.com/apache/spark/pull/7204#issuecomment-118707328
  
@shivaram Thank you for merging it!
I agree with that. I will check the lintr code and ask Jim 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: [SPARK-8549][SparkR] Fix the line length of Sp...

2015-07-05 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-8549][SparkR] Fix the line length of Sp...

2015-07-05 Thread shivaram
Github user shivaram commented on the pull request:

https://github.com/apache/spark/pull/7204#issuecomment-118706866
  
Thanks @yu-iskw. BTW We should investigate how to get rid of the false 
warnings `no visible global function definition` as those functions are package 
private.

But this change LGTM. Merging 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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7219#issuecomment-118705402
  
  [Test build #36556 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36556/consoleFull)
 for   PR 7219 at commit 
[`537579a`](https://github.com/apache/spark/commit/537579ad94036cb392d82d3802eccc38d72fb861).


---
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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7219#issuecomment-118703671
  
 Merged build triggered.


---
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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7219#issuecomment-118703711
  
Merged build started.


---
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: [SPARK-8389][Streaming][PySpark] Expose KafkaR...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7185#issuecomment-118703589
  
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 pull request: [SPARK-8389][Streaming][PySpark] Expose KafkaR...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7185#issuecomment-118703559
  
  [Test build #36555 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36555/console)
 for   PR 7185 at commit 
[`fd13937`](https://github.com/apache/spark/commit/fd13937ec28f0f01e410c5247859fde1f15b6a61).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class KafkaRDD(RDD):`
  * `class KafkaDStream(DStream):`



---
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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread andrewor14
Github user andrewor14 commented on the pull request:

https://github.com/apache/spark/pull/7219#issuecomment-118703366
  
@srowen @pwendell please let me know whether I understood your comments 
correctly in the latest commit.


---
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: [SPARK-8819] Fix build for maven 3.3.x

2015-07-05 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/7219#discussion_r33904037
  
--- Diff: pom.xml ---
@@ -1826,6 +1830,26 @@
   
 
 
+
+  
+  release
--- End diff --

I still need to add it to the `create-release.sh` script though right?


---
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: [SPARK-8221][SQL]Add pmod function

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/6783#issuecomment-118700999
  
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 pull request: [SPARK-8221][SQL]Add pmod function

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/6783#issuecomment-118700970
  
  [Test build #36552 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36552/console)
 for   PR 6783 at commit 
[`cbaf9f5`](https://github.com/apache/spark/commit/cbaf9f5efecf152b1bda69d3f4493befd9fb1f8a).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class Pmod(left: Expression, right: Expression) extends 
BinaryArithmetic `



---
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: [SPARK-8271][SQL]string function: soundex

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7115#issuecomment-118700840
  
  [Test build #999 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/999/console)
 for   PR 7115 at commit 
[`f88d320`](https://github.com/apache/spark/commit/f88d3202f65821159c7b750fbfc003b07f68e993).
 * This patch **fails Spark unit tests**.
 * This patch **does not merge cleanly**.
 * This patch adds the following public classes _(experimental)_:
  * `class Word2VecModel(JavaVectorTransformer, JavaSaveable, JavaLoader):`
  * `case class CreateNamedStruct(children: Seq[Expression]) extends 
Expression `
  * `case class Factorial(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class ShiftLeft(left: Expression, right: Expression) extends 
BinaryExpression `
  * `case class ShiftRight(left: Expression, right: Expression) extends 
BinaryExpression `
  * `case class Md5(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Sha1(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Crc32(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Not(child: Expression) extends UnaryExpression with 
Predicate with ExpectsInputTypes `
  * `trait StringRegexExpression extends ExpectsInputTypes `
  * `trait CaseConversionExpression extends ExpectsInputTypes `
  * `trait StringComparison extends ExpectsInputTypes `
  * `case class StringLength(child: Expression) extends UnaryExpression 
with ExpectsInputTypes `
  * `case class SoundEx(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `protected[sql] abstract class AtomicType extends DataType `
  * `abstract class NumericType extends AtomicType `
  * `abstract class DataType extends AbstractDataType `



---
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: [SPARK-8389][Streaming][PySpark] Expose KafkaR...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7185#issuecomment-118700156
  
  [Test build #36555 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36555/consoleFull)
 for   PR 7185 at commit 
[`fd13937`](https://github.com/apache/spark/commit/fd13937ec28f0f01e410c5247859fde1f15b6a61).


---
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: [SPARK-8811][SQL] Read array struct data from ...

2015-07-05 Thread Sephiroth-Lin
Github user Sephiroth-Lin commented on the pull request:

https://github.com/apache/spark/pull/7209#issuecomment-118699916
  
@liancheng OK, good, thank you.


---
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: [SPARK-8389][Streaming][PySpark] Expose KafkaR...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7185#issuecomment-118699892
  
Merged build started.


---
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: [SPARK-8389][Streaming][PySpark] Expose KafkaR...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7185#issuecomment-118699883
  
 Merged build triggered.


---
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: [SPARK-8233][SQL] misc function: hash

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/6971#issuecomment-118699528
  
  [Test build #998 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/998/console)
 for   PR 6971 at commit 
[`d79088b`](https://github.com/apache/spark/commit/d79088b8eb733bdc61e3a9cbc956b1eddaef26c0).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class Word2VecModel(JavaVectorTransformer, JavaSaveable, JavaLoader):`
  * `case class ScalaUDF(`
  * `case class CreateNamedStruct(children: Seq[Expression]) extends 
Expression `
  * `case class CurrentDate() extends LeafExpression `
  * `case class CurrentTimestamp() extends LeafExpression `
  * `case class Factorial(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Hex(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class UnHex(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class ShiftLeft(left: Expression, right: Expression)`
  * `case class ShiftRight(left: Expression, right: Expression)`
  * `case class ShiftRightUnsigned(left: Expression, right: Expression)`
  * `case class Md5(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Hash(children: Expression*) extends Expression `
  * `case class Sha1(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Crc32(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Not(child: Expression) extends UnaryExpression with 
Predicate with ExpectsInputTypes `
  * `trait StringRegexExpression extends ExpectsInputTypes `
  * `trait CaseConversionExpression extends ExpectsInputTypes `
  * `trait StringComparison extends ExpectsInputTypes `
  * `case class StringLength(child: Expression) extends UnaryExpression 
with ExpectsInputTypes `
  * `case class Levenshtein(left: Expression, right: Expression) extends 
BinaryExpression`
  * `case class Ascii(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Base64(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class UnBase64(child: Expression) extends UnaryExpression with 
ExpectsInputTypes `
  * `case class Decode(bin: Expression, charset: Expression) extends 
Expression with ExpectsInputTypes `
  * `case class Encode(value: Expression, charset: Expression)`
  * `protected[sql] abstract class AtomicType extends DataType `
  * `abstract class NumericType extends AtomicType `
  * `abstract class DataType extends AbstractDataType `
  * `case class UserDefinedFunction protected[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: [SPARK-8271][SQL]string function: soundex

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7115#issuecomment-118699444
  
  [Test build #999 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/999/consoleFull)
 for   PR 7115 at commit 
[`f88d320`](https://github.com/apache/spark/commit/f88d3202f65821159c7b750fbfc003b07f68e993).


---
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: [SPARK-8233][SQL] misc function: hash

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/6971#issuecomment-118699427
  
  [Test build #998 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/998/consoleFull)
 for   PR 6971 at commit 
[`d79088b`](https://github.com/apache/spark/commit/d79088b8eb733bdc61e3a9cbc956b1eddaef26c0).


---
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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118698676
  
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 pull request: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118698670
  
  [Test build #36554 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36554/console)
 for   PR 7231 at commit 
[`9b87903`](https://github.com/apache/spark/commit/9b87903f3e6621ce19c49135f996b1267b233a08).
 * This patch **fails MiMa 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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118695116
  
  [Test build #36554 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/36554/consoleFull)
 for   PR 7231 at commit 
[`9b87903`](https://github.com/apache/spark/commit/9b87903f3e6621ce19c49135f996b1267b233a08).


---
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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118694192
  
 Merged build triggered.


---
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: [SPARK-6776] [SPARK-8811] [SQL] Refactors Parq...

2015-07-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7231#issuecomment-118694202
  
Merged build started.


---
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: [Spark-8703] [ML] Add CountVectorizer as a ml ...

2015-07-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/7084#discussion_r33902864
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala ---
@@ -0,0 +1,79 @@
+/*
+ * 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.ml.feature
+
+import scala.collection.mutable
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.ml.UnaryTransformer
+import org.apache.spark.ml.param.{ParamMap, ParamValidators, IntParam}
+import org.apache.spark.ml.util.Identifiable
+import org.apache.spark.mllib.linalg.{Vectors, VectorUDT, Vector}
+import org.apache.spark.sql.types.{StringType, ArrayType, DataType}
+
+/**
+ * :: Experimental ::
+ * Converts a text document to a sparse vector of token counts.
+ * @param vocabulary An Array over terms. Only the terms in the vocabulary 
will be counted.
+ */
+@Experimental
+class CountVectorizer (override val uid: String, vocabulary: Array[String])
+  extends UnaryTransformer[Seq[String], Vector, CountVectorizer] {
+
+  def this(vocabulary: Array[String]) = 
this(Identifiable.randomUID("countVectorizer"), vocabulary)
+
+  /**
+   * Corpus-specific stop words filter. Terms with count less than the 
given threshold are ignored.
+   * Default: 1
+   * @group param
+   */
+  val minTermCounts: IntParam = new IntParam(this, "minTermCounts",
--- End diff --

Thanks for the careful consideration.


---
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   >