[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199286570
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingLimitExec.scala
 ---
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.streaming
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
Distribution, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.streaming.state.StateStoreOps
+import org.apache.spark.sql.types.{LongType, NullType, StructField, 
StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * A physical operator for executing a streaming limit, which makes sure 
no more than streamLimit
+ * rows are returned.
+ */
+case class StreamingLimitExec(
+streamLimit: Long,
+child: SparkPlan,
+stateInfo: Option[StatefulOperatorStateInfo] = None)
+  extends UnaryExecNode with StateStoreWriter {
+
+  private val keySchema = StructType(Array(StructField("key", NullType)))
+  private val valueSchema = StructType(Array(StructField("value", 
LongType)))
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics // force lazy init at driver
+
+child.execute().mapPartitionsWithStateStore(
+  getStateInfo,
+  keySchema,
+  valueSchema,
+  indexOrdinal = None,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
+  val key = UnsafeProjection.create(keySchema)(new 
GenericInternalRow(Array[Any](null)))
+  val numOutputRows = longMetric("numOutputRows")
+  val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+  val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+  val commitTimeMs = longMetric("commitTimeMs")
+  val updatesStartTimeNs = System.nanoTime
+
+  val startCount: Long = 
Option(store.get(key)).map(_.getLong(0)).getOrElse(0L)
+  var rowCount = startCount
+
+  val result = iter.filter { r =>
+val x = rowCount < streamLimit
--- End diff --

Oh and we should be planning a `LocalLimit` before this and perhaps 
`GlobalStreamingLimitExec` would be a better name to make the functionality 
obvious.


---

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



[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199286349
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingLimitExec.scala
 ---
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.streaming
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
Distribution, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.streaming.state.StateStoreOps
+import org.apache.spark.sql.types.{LongType, NullType, StructField, 
StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * A physical operator for executing a streaming limit, which makes sure 
no more than streamLimit
+ * rows are returned.
+ */
+case class StreamingLimitExec(
+streamLimit: Long,
+child: SparkPlan,
+stateInfo: Option[StatefulOperatorStateInfo] = None)
+  extends UnaryExecNode with StateStoreWriter {
+
+  private val keySchema = StructType(Array(StructField("key", NullType)))
+  private val valueSchema = StructType(Array(StructField("value", 
LongType)))
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics // force lazy init at driver
+
+child.execute().mapPartitionsWithStateStore(
+  getStateInfo,
+  keySchema,
+  valueSchema,
+  indexOrdinal = None,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
+  val key = UnsafeProjection.create(keySchema)(new 
GenericInternalRow(Array[Any](null)))
+  val numOutputRows = longMetric("numOutputRows")
+  val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+  val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+  val commitTimeMs = longMetric("commitTimeMs")
+  val updatesStartTimeNs = System.nanoTime
+
+  val startCount: Long = 
Option(store.get(key)).map(_.getLong(0)).getOrElse(0L)
+  var rowCount = startCount
+
+  val result = iter.filter { r =>
+val x = rowCount < streamLimit
--- End diff --

I think its okay due to `override def requiredChildDistribution: 
Seq[Distribution] = AllTuples :: Nil`.

+1 to making sure there are tests with more than one partition though.



---

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



[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199279042
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingLimitExec.scala
 ---
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.streaming
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
Distribution, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.streaming.state.StateStoreOps
+import org.apache.spark.sql.types.{LongType, NullType, StructField, 
StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * A physical operator for executing a streaming limit, which makes sure 
no more than streamLimit
+ * rows are returned.
+ */
+case class StreamingLimitExec(
+streamLimit: Long,
+child: SparkPlan,
+stateInfo: Option[StatefulOperatorStateInfo] = None)
+  extends UnaryExecNode with StateStoreWriter {
+
+  private val keySchema = StructType(Array(StructField("key", NullType)))
+  private val valueSchema = StructType(Array(StructField("value", 
LongType)))
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics // force lazy init at driver
--- End diff --

Existing: Do we really do this in every operator?  Why isn't this the 
responsibility of the parent class?


---

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



[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199280992
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala ---
@@ -615,7 +615,7 @@ class StreamSuite extends StreamTest {
 // Get an existing checkpoint generated by Spark v2.1.
 // v2.1 does not record # shuffle partitions in the offset metadata.
 val resourceUri =
-  
this.getClass.getResource("/structured-streaming/checkpoint-version-2.1.0").toURI
+
this.getClass.getResource("/structured-streaming/checkpoint-version-2.1.0").toURI
--- End diff --

I'd undo these spurious changes.


---

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



[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199278369
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala ---
@@ -72,6 +72,8 @@ abstract class SparkStrategies extends 
QueryPlanner[SparkPlan] {
 case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, 
true, child)))
 if limit < conf.topKSortFallbackThreshold =>
   TakeOrderedAndProjectExec(limit, order, projectList, 
planLater(child)) :: Nil
+case Limit(IntegerLiteral(limit), child) if plan.isStreaming =>
+  StreamingLimitExec(limit, planLater(child)) :: Nil
--- End diff --

I would create a different one only to continue the pattern of isolating 
streaming specific Strategies.  You'll then need to inject your new Strategy in 
`IncrementalExecution`.


---

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



[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199278862
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingLimitExec.scala
 ---
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.streaming
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
Distribution, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.streaming.state.StateStoreOps
+import org.apache.spark.sql.types.{LongType, NullType, StructField, 
StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * A physical operator for executing a streaming limit, which makes sure 
no more than streamLimit
+ * rows are returned.
+ */
+case class StreamingLimitExec(
+streamLimit: Long,
+child: SparkPlan,
+stateInfo: Option[StatefulOperatorStateInfo] = None)
+  extends UnaryExecNode with StateStoreWriter {
+
+  private val keySchema = StructType(Array(StructField("key", NullType)))
+  private val valueSchema = StructType(Array(StructField("value", 
LongType)))
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics // force lazy init at driver
+
+child.execute().mapPartitionsWithStateStore(
+  getStateInfo,
+  keySchema,
+  valueSchema,
+  indexOrdinal = None,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
+  val key = UnsafeProjection.create(keySchema)(new 
GenericInternalRow(Array[Any](null)))
+  val numOutputRows = longMetric("numOutputRows")
+  val numUpdatedStateRows = longMetric("numUpdatedStateRows")
+  val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+  val commitTimeMs = longMetric("commitTimeMs")
+  val updatesStartTimeNs = System.nanoTime
+
+  val startCount: Long = 
Option(store.get(key)).map(_.getLong(0)).getOrElse(0L)
+  var rowCount = startCount
+
+  val result = iter.filter { r =>
+val x = rowCount < streamLimit
+if (x) {
+  rowCount += 1
+}
+x
+  }
+
+  CompletionIterator[InternalRow, Iterator[InternalRow]](result, {
--- End diff --

Do you need these type parameters?


---

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



[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199279216
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingLimitExec.scala
 ---
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.streaming
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
Distribution, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.streaming.state.StateStoreOps
+import org.apache.spark.sql.types.{LongType, NullType, StructField, 
StructType}
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * A physical operator for executing a streaming limit, which makes sure 
no more than streamLimit
+ * rows are returned.
+ */
+case class StreamingLimitExec(
+streamLimit: Long,
+child: SparkPlan,
+stateInfo: Option[StatefulOperatorStateInfo] = None)
+  extends UnaryExecNode with StateStoreWriter {
+
+  private val keySchema = StructType(Array(StructField("key", NullType)))
+  private val valueSchema = StructType(Array(StructField("value", 
LongType)))
+
+  override protected def doExecute(): RDD[InternalRow] = {
+metrics // force lazy init at driver
+
+child.execute().mapPartitionsWithStateStore(
+  getStateInfo,
+  keySchema,
+  valueSchema,
+  indexOrdinal = None,
+  sqlContext.sessionState,
+  Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
--- End diff --

Nit: I'd indent 4 above to distinguish these two blocks visually.


---

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



[GitHub] spark pull request #21662: [SPARK-24662][SQL][SS] Support limit in structure...

2018-06-29 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/21662#discussion_r199278041
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
 ---
@@ -315,8 +315,10 @@ object UnsupportedOperationChecker {
 case GroupingSets(_, _, child, _) if child.isStreaming =>
   throwError("GroupingSets is not supported on streaming 
DataFrames/Datasets")
 
-case GlobalLimit(_, _) | LocalLimit(_, _) if 
subPlan.children.forall(_.isStreaming) =>
-  throwError("Limits are not supported on streaming 
DataFrames/Datasets")
+case GlobalLimit(_, _) | LocalLimit(_, _) if
+  subPlan.children.forall(_.isStreaming) && outputMode == 
InternalOutputModes.Update =>
--- End diff --

It is today (though as we discussed I think the query planner would be a 
better place if we were to rearchitect).

Style nit: line break at the high syntactic level (i.e. before the if) and 
indent 4 space for a continuation like this (to distinguish the guard from the 
code executed when matched.


---

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



[GitHub] spark pull request #20647: [SPARK-23303][SQL] improve the explain result for...

2018-02-23 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/20647#discussion_r170185948
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
 ---
@@ -77,31 +79,32 @@ class MicroBatchExecution(
   
sparkSession.sqlContext.conf.disabledV2StreamingMicroBatchReaders.split(",")
 
 val _logicalPlan = analyzedPlan.transform {
-  case streamingRelation@StreamingRelation(dataSourceV1, sourceName, 
output) =>
-toExecutionRelationMap.getOrElseUpdate(streamingRelation, {
+  case s @ StreamingRelation(dsV1, sourceName, output) =>
--- End diff --

If you are touching that specific code then its fine to fix the style, but 
in general I tend to agree that it makes the diff harder to read and commit 
harder to back port if you include spurious changes.

I've even seen guidelines that specifically prohibit fixing style just to 
fix style since it obfuscates the history.


---

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



[GitHub] spark pull request #20647: [SPARK-23303][SQL] improve the explain result for...

2018-02-22 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/20647#discussion_r170115965
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
 ---
@@ -415,12 +418,14 @@ class MicroBatchExecution(
 case v1: SerializedOffset => reader.deserializeOffset(v1.json)
 case v2: OffsetV2 => v2
   }
-  reader.setOffsetRange(
-toJava(current),
-Optional.of(availableV2))
+  reader.setOffsetRange(toJava(current), Optional.of(availableV2))
   logDebug(s"Retrieving data from $reader: $current -> 
$availableV2")
-  Some(reader ->
-new 
StreamingDataSourceV2Relation(reader.readSchema().toAttributes, reader))
+  Some(reader -> StreamingDataSourceV2Relation(
--- End diff --

@rdblue There was a doc as part of this SPIP: 
https://issues.apache.org/jira/browse/SPARK-20928, but it has definitely 
evolved enough past that we should update and send to the dev list again.

Things like the logical plan requirement in execution will likely be 
significantly easier to remove once we have a full V2 API and can remove the 
legacy internal API for streaming.


---

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



[GitHub] spark issue #20598: [SPARK-23406] [SS] Enable stream-stream self-joins

2018-02-21 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20598
  
Yeah, this seems risky at RC5.


---

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



[GitHub] spark pull request #20647: [SPARK-23303][SQL] improve the explain result for...

2018-02-21 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/20647#discussion_r169799465
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 ---
@@ -107,17 +106,24 @@ case class DataSourceV2Relation(
 }
 
 /**
- * A specialization of DataSourceV2Relation with the streaming bit set to 
true. Otherwise identical
- * to the non-streaming relation.
+ * A specialization of [[DataSourceV2Relation]] with the streaming bit set 
to true.
+ *
+ * Note that, this plan has a mutable reader, so Spark won't apply 
operator push-down for this plan,
+ * to avoid making the plan mutable. We should consolidate this plan and 
[[DataSourceV2Relation]]
+ * after we figure out how to apply operator push-down for streaming data 
sources.
--- End diff --

@rdblue as well.

I agree that we really need to define the contract here, and personally I 
would really benefit from seeing a life cycle diagram for all of the different 
pieces of the API. @tdas and @jose-torres made one for just the write side of 
streaming and it was super useful for me as someone at a distance that wants to 
understand what was going on.

![screen shot 2018-02-21 at 2 18 55 
pm](https://user-images.githubusercontent.com/527/36508744-65d25be0-1712-11e8-93c6-52515f8b50e9.png)

Something like this diagram that also covered when things are resolved, 
when pushdown happens, and that shows the differences between read/write, 
microbatch, batch and continuous would be awesome.

Regarding the actual question, I'm not a huge fan of option 2 as it still 
seems like an implicit contract with this mutable object (assuming I understand 
the proposal correctly).  Option 1 at least means that we could say, "whenever 
its time to do pushdown: call `reset()`, do pushdown in some defined order, 
then call `createX()`.  It is invalid to do more pushdown after createX has 
been called".

Even better than a `reset()` might be a `cleanClone()` method that gives 
you a fresh copy. As I said above, I don't really understand the lifecycle of 
the API, but given how we reuse query plan fragments I'm really nervous about 
mutable objects that are embedded in operators.

I also agree with @jose-torres point that this mechanism looks like action 
at a distance, but the `reset()` contract at least localizes it to some degree, 
and I don't have a better suggestion for a way to support evolvable pushdown. 



---

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



[GitHub] spark issue #20511: [SPARK-23340][SQL] Upgrade Apache ORC to 1.4.3

2018-02-14 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20511
  
Unfortunately, dependency changes are not typically allowed in patch 
releases.


---

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



[GitHub] spark pull request #20598: [SPARK-23406] [SS] Enable stream-stream self-join...

2018-02-13 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/20598#discussion_r168011671
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
 ---
@@ -431,7 +431,11 @@ class MicroBatchExecution(
 s"Invalid batch: ${Utils.truncatedString(output, ",")} != " +
   s"${Utils.truncatedString(dataPlan.output, ",")}")
   replacements ++= output.zip(dataPlan.output)
--- End diff --

I think this is no longer used.


---

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



[GitHub] spark pull request #20598: [SPARK-23406] [SS] Enable stream-stream self-join...

2018-02-13 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/20598#discussion_r168011940
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
 ---
@@ -440,8 +444,6 @@ class MicroBatchExecution(
 // Rewire the plan to use the new attributes that were returned by the 
source.
 val replacementMap = AttributeMap(replacements)
--- End diff --

Same


---

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



[GitHub] spark issue #20511: [SPARK-23340][SQL] Upgrade Apache ORC to 1.4.3

2018-02-13 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20511
  
Sorry if I'm missing some context here, but our typical process this late 
in the release (we are over a month since the branch was cut) would be to 
disable any new features that still have regressions.  I'm generally not okay 
with any dependency changes this far into QA unless there is no other option.

Should we be considering turning this off so that we can GA Spark 2.3 soon?


---

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



[GitHub] spark issue #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable logical ...

2018-02-02 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20387
  
Regarding, `computeStats`, the logical plan seems like it might not be the 
right place.  As we move towards more CBO it seems like we are going to need to 
pick physical operators before we can really reason about the cost of a 
subplan. With the caveat that I haven't though hard about this, I'd be 
supportive of moving these kinds of metrics to physical plan. +1 that we need 
to be able to consider pushdown when producing stats either way.

On the second point, I don't think I understand DataSourceV2 enough yet to 
know the answer, but you ask a lot of questions that I think need to be defined 
as part of the API (if we haven't already).  What is the contract for ordering 
and interactions between different types of pushdown? Is it valid to pushdown 
in pieces or will we only call the method once? (sorry if this is written down 
and I've just missed it).

My gut feeling is that we don't really want to fuse incrementally.  Its 
seems hard to reason about correctness and interactions between different 
things that have been pushed.  As I hinted at before, I think its most natural 
to split the concerns of pushdown within a query plan and fusing of operators. 
But maybe this is limited in someway I don't realize.


---

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



[GitHub] spark pull request #20387: [SPARK-23203][SQL]: DataSourceV2: Use immutable l...

2018-02-02 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/20387#discussion_r165751660
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 ---
@@ -17,15 +17,149 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import java.util.UUID
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
Expression}
 import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
+import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, 
ReadSupport, ReadSupportWithSchema, WriteSupport}
 import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.writer.DataSourceV2Writer
+import org.apache.spark.sql.types.StructType
 
 case class DataSourceV2Relation(
-fullOutput: Seq[AttributeReference],
-reader: DataSourceV2Reader) extends LeafNode with 
DataSourceReaderHolder {
+source: DataSourceV2,
+options: Map[String, String],
+path: Option[String] = None,
+table: Option[TableIdentifier] = None,
+projection: Option[Seq[AttributeReference]] = None,
+filters: Option[Seq[Expression]] = None,
--- End diff --

I like this pattern.  I think it is important that the arguments to a query 
plan node are comprehensive so that it is easy to understand what is going on 
in the output of `explain()`.


---

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



[GitHub] spark issue #20085: [SPARK-22739][Catalyst][WIP] Additional Expression Suppo...

2018-01-14 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20085
  
This blocks better support for encoders on spark-avro, and seems safe, so 
I'd really like to include it in possible.


---

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



[GitHub] spark issue #20219: [SPARK-23025][SQL] Support Null type in scala reflection

2018-01-11 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20219
  
This seems reasonable to me.  You can already do `sql("SELECT null")`


---

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



[GitHub] spark issue #20010: [SPARK-22826][SQL] findWiderTypeForTwo Fails over Struct...

2018-01-03 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20010
  
/cc @cloud-fan @sameeragarwal 


---

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



[GitHub] spark issue #20085: [SPARK-22739][Catalyst][WIP] Additional Expression Suppo...

2018-01-03 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/20085
  
/cc @cloud-fan @sameeragarwal 


---

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



[GitHub] spark issue #16578: [SPARK-4502][SQL] Parquet nested column pruning

2018-01-03 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/16578
  
I agree that this PR needs to be allocated more review bandwidth, and it is 
unfortunate that it has been blocked on that. However, I am -1 on merging a 
change this large after branch cut.


---

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



[GitHub] spark pull request #20048: [SPARK-22862] Docs on lazy elimination of columns...

2017-12-21 Thread marmbrus
GitHub user marmbrus opened a pull request:

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

[SPARK-22862] Docs on lazy elimination of columns missing from an encoder

This behavior has confused some users, so lets clarify it.

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

$ git pull https://github.com/marmbrus/spark datasetAsDocs

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

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


commit 2f4e7b04f8d62d73225f00c12a95136c556a15d7
Author: Michael Armbrust <michael@...>
Date:   2017-12-21T19:48:46Z

[SPARK-22862] Docs on lazy elimination of columns missing from an encoder




---

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



[GitHub] spark issue #19925: [SPARK-22732] Add Structured Streaming APIs to DataSourc...

2017-12-14 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/19925
  
I would probably do ... streaming.reader/writer if we are going to 
namespace it.


---

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



[GitHub] spark issue #19771: [SPARK-22544][SS]FileStreamSource should use its own had...

2017-11-17 Thread marmbrus
Github user marmbrus commented on the issue:

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


---

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



[GitHub] spark issue #19461: [SPARK-22230] Swap per-row order in state store restore.

2017-10-09 Thread marmbrus
Github user marmbrus commented on the issue:

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


---

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



[GitHub] spark issue #19314: [SPARK-22094][SS]processAllAvailable should check the qu...

2017-09-21 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/19314
  
LGTM!


---

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



[GitHub] spark issue #19240: [SPARK-22018][SQL]Preserve top-level alias metadata when...

2017-09-14 Thread marmbrus
Github user marmbrus commented on the issue:

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


---

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



[GitHub] spark pull request #19240: [SPARK-22018][SQL]Preserve top-level alias metada...

2017-09-14 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/19240#discussion_r139042816
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -2256,7 +2256,10 @@ object CleanupAliases extends Rule[LogicalPlan] {
 
   def trimNonTopLevelAliases(e: Expression): Expression = e match {
 case a: Alias =>
-  a.withNewChildren(trimAliases(a.child) :: Nil)
--- End diff --

Is there anywhere else in the code base where we call `withNewChildren` on 
an alias?  Should we just override that?


---

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



[GitHub] spark pull request #19080: [SPARK-21865][SQL] simplify the distribution sema...

2017-08-30 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/19080#discussion_r136213945
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 ---
@@ -30,18 +30,43 @@ import org.apache.spark.sql.types.{DataType, 
IntegerType}
  *  - Intra-partition ordering of data: In this case the distribution 
describes guarantees made
  *about how tuples are distributed within a single partition.
  */
-sealed trait Distribution
+sealed trait Distribution {
+  /**
+   * The required number of partitions for this distribution. If it's 
None, then any number of
+   * partitions is allowed for this distribution.
+   */
+  def requiredNumPartitions: Option[Int]
+
+  /**
+   * Creates a default partitioning for this distribution, which can 
satisfy this distribution while
+   * matching the given number of partitions.
+   */
+  def createPartitioning(numPartitions: Int): Partitioning
+}
 
 /**
  * Represents a distribution where no promises are made about co-location 
of data.
  */
-case object UnspecifiedDistribution extends Distribution
+case object UnspecifiedDistribution extends Distribution {
+  override def requiredNumPartitions: Option[Int] = None
+
+  override def createPartitioning(numPartitions: Int): Partitioning = {
+throw new IllegalStateException("UnspecifiedDistribution does not have 
default partitioning.")
+  }
+}
 
 /**
  * Represents a distribution that only has a single partition and all 
tuples of the dataset
  * are co-located.
  */
-case object AllTuples extends Distribution
+case object AllTuples extends Distribution {
--- End diff --

It seems we should be moving towards CBO, not away from 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 #19080: [SPARK-21865][SQL] simplify the distribution sema...

2017-08-30 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/19080#discussion_r136213879
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 ---
@@ -30,18 +30,43 @@ import org.apache.spark.sql.types.{DataType, 
IntegerType}
  *  - Intra-partition ordering of data: In this case the distribution 
describes guarantees made
  *about how tuples are distributed within a single partition.
  */
-sealed trait Distribution
+sealed trait Distribution {
+  /**
+   * The required number of partitions for this distribution. If it's 
None, then any number of
+   * partitions is allowed for this distribution.
+   */
+  def requiredNumPartitions: Option[Int]
+
+  /**
+   * Creates a default partitioning for this distribution, which can 
satisfy this distribution while
+   * matching the given number of partitions.
+   */
+  def createPartitioning(numPartitions: Int): Partitioning
+}
 
 /**
  * Represents a distribution where no promises are made about co-location 
of data.
  */
-case object UnspecifiedDistribution extends Distribution
+case object UnspecifiedDistribution extends Distribution {
+  override def requiredNumPartitions: Option[Int] = None
+
+  override def createPartitioning(numPartitions: Int): Partitioning = {
+throw new IllegalStateException("UnspecifiedDistribution does not have 
default partitioning.")
+  }
+}
 
 /**
  * Represents a distribution that only has a single partition and all 
tuples of the dataset
  * are co-located.
  */
-case object AllTuples extends Distribution
+case object AllTuples extends Distribution {
--- End diff --

Sure you can make that argument (I'm not sure I buy it), but then why does 
this PR still have two different concepts?


---
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 #19080: [SPARK-21865][SQL] simplify the distribution sema...

2017-08-30 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/19080#discussion_r136209980
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 ---
@@ -30,18 +30,43 @@ import org.apache.spark.sql.types.{DataType, 
IntegerType}
  *  - Intra-partition ordering of data: In this case the distribution 
describes guarantees made
  *about how tuples are distributed within a single partition.
  */
-sealed trait Distribution
+sealed trait Distribution {
+  /**
+   * The required number of partitions for this distribution. If it's 
None, then any number of
+   * partitions is allowed for this distribution.
+   */
+  def requiredNumPartitions: Option[Int]
+
+  /**
+   * Creates a default partitioning for this distribution, which can 
satisfy this distribution while
+   * matching the given number of partitions.
+   */
+  def createPartitioning(numPartitions: Int): Partitioning
+}
 
 /**
  * Represents a distribution where no promises are made about co-location 
of data.
  */
-case object UnspecifiedDistribution extends Distribution
+case object UnspecifiedDistribution extends Distribution {
+  override def requiredNumPartitions: Option[Int] = None
+
+  override def createPartitioning(numPartitions: Int): Partitioning = {
+throw new IllegalStateException("UnspecifiedDistribution does not have 
default partitioning.")
+  }
+}
 
 /**
  * Represents a distribution that only has a single partition and all 
tuples of the dataset
  * are co-located.
  */
-case object AllTuples extends Distribution
+case object AllTuples extends Distribution {
--- End diff --

So I spoke with @sameeragarwal about this a little.  The whole point here 
was to have a logical / physical separation.  `AllTuples` could be `SingleNode` 
or it could be `Broadcast`.  All the operation wants to know is that its seeing 
all of them and it shouldn't care about how that is being accomplished.

Now, since the first version, we have started to deviate from that.  I'm 
not sure if this is still the right thing to do, but I wanted to give a little 
context.


---
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 #18923: [SPARK-21710][StSt] Fix OOM on ConsoleSink with l...

2017-08-11 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/18923#discussion_r132801760
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala 
---
@@ -49,7 +49,7 @@ class ConsoleSink(options: Map[String, String]) extends 
Sink with Logging {
 println("---")
 // scalastyle:off println
 data.sparkSession.createDataFrame(
-  data.sparkSession.sparkContext.parallelize(data.collect()), 
data.schema)
--- End diff --

I don't think this means we can't do anything.  I just think that we need 
to fix the query plan and call take without changing the plan.  Its kind of a 
hack but it would work until we make the planner smarter.

I think something like `data.queryExecution.executedPlan.executeTake(...)` 
would be safe.


---
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 #18923: [SPARK-21710][StSt] Fix OOM on ConsoleSink with l...

2017-08-11 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/18923#discussion_r132792708
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala 
---
@@ -49,7 +49,7 @@ class ConsoleSink(options: Map[String, String]) extends 
Sink with Logging {
 println("---")
 // scalastyle:off println
 data.sparkSession.createDataFrame(
-  data.sparkSession.sparkContext.parallelize(data.collect()), 
data.schema)
--- End diff --

This might not be safe unfortunately.  Anything that changes the query plan 
might cause it to get replanned (and we don't do this correctly).

Can you make sure that there is a test case that uses the console sink and 
does aggregation across more than one batch?


---
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 #18925: [SPARK-21713][SC] Replace streaming bit with Outp...

2017-08-11 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/18925#discussion_r132791950
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -779,10 +780,16 @@ case object OneRowRelation extends LeafNode {
 }
 
 /** A logical plan for `dropDuplicates`. */
+case object Deduplicate {
+  def apply(keys: Seq[Attribute], child: LogicalPlan): Deduplicate = {
+Deduplicate(keys, child, child.outputMode)
+  }
+}
+
 case class Deduplicate(
 keys: Seq[Attribute],
 child: LogicalPlan,
-streaming: Boolean) extends UnaryNode {
+originalOutputMode: OutputMode) extends UnaryNode {
--- End diff --

Can we drop this?  Can it just preserve the output mode of its child?


---
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 #18925: [SPARK-21713][SC] Replace streaming bit with Outp...

2017-08-11 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/18925#discussion_r132791711
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1185,7 +1186,7 @@ object ReplaceDistinctWithAggregate extends 
Rule[LogicalPlan] {
  */
 object ReplaceDeduplicateWithAggregate extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
-case Deduplicate(keys, child, streaming) if !streaming =>
+case d @ Deduplicate(keys, child, _) if d.originalOutputMode != 
OutputMode.Append() =>
--- End diff --

existing: It's kind of odd that this decision is made in the optimizer and 
not the query planner.  I think our aggregate operator is actually worse than 
the specialized deduplication operator (since the specialized one is 
non-blocking).

It doesn't have to be in this PR, but we should probably move this to the 
planner eventually.


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

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



[GitHub] spark issue #18925: [SPARK-21713][SC] Replace streaming bit with OutputMode

2017-08-11 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/18925
  
ok to 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 issue #18925: [SPARK-21713][SC] Replace streaming bit with OutputMode

2017-08-11 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/18925
  
add to whitelist


---
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 #18828: [SPARK-21619][SQL] Fail the execution of canonica...

2017-08-03 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/18828#discussion_r131250896
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala 
---
@@ -181,17 +181,38 @@ abstract class QueryPlan[PlanType <: 
QueryPlan[PlanType]] extends TreeNode[PlanT
   override def innerChildren: Seq[QueryPlan[_]] = subqueries
 
   /**
+   * A private mutable variable to indicate whether this plan is the 
result of canonicalization.
+   * This is used solely for making sure we wouldn't execute a 
canonicalized plan.
+   * See [[canonicalized]] on how this is set.
+   */
+  @transient
--- End diff --

I guess plans are already not valid on executors, by why `@transient`?


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

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



[GitHub] spark issue #18822: [SPARK-21546][SS] dropDuplicates should ignore watermark...

2017-08-02 Thread marmbrus
Github user marmbrus commented on the issue:

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


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

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



[GitHub] spark issue #18803: [SPARK-21597][SS]Fix a potential overflow issue in Event...

2017-08-01 Thread marmbrus
Github user marmbrus commented on the issue:

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


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

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



[GitHub] spark issue #18780: [INTRA] Close stale PRs

2017-07-31 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/18780
  
I'm in favor of fairly aggressive closing of inactive pull requests. The 
cost of reopening them is small, and the benefits of having a clear view of 
what is in progress are large. I think as long as the notification is clear and 
polite, it won't discourage contribution.


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

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



[GitHub] spark issue #18658: [SPARK-20871][SQL] only log Janino code at debug level

2017-07-17 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/18658
  
That seems reasonable.  I'm kind of pro-truncation for very very large 
code.  Even though its not great to have something truncated, outputting GBs of 
logs is also pretty bad for downstream consumers.


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

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



[GitHub] spark issue #18658: [SPARK-20871][SQL] only log Janino code at debug level

2017-07-17 Thread marmbrus
Github user marmbrus commented on the issue:

https://github.com/apache/spark/pull/18658
  
I don't have super strong opinions here, but in my experience its not 
always easy to get users to rerun a failed query with a different logging 
level.  Have we considered truncating or special casing the 64k limitation 
instead?


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

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



[GitHub] spark issue #18638: [SPARK-21421][SS]Add the query id as a local property to...

2017-07-14 Thread marmbrus
Github user marmbrus commented on the issue:

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



spark-website git commit: Add pyspark instructions

2017-07-13 Thread marmbrus
Repository: spark-website
Updated Branches:
  refs/heads/asf-site 5ddf243fd -> f57f0702b


Add pyspark instructions


Project: http://git-wip-us.apache.org/repos/asf/spark-website/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark-website/commit/f57f0702
Tree: http://git-wip-us.apache.org/repos/asf/spark-website/tree/f57f0702
Diff: http://git-wip-us.apache.org/repos/asf/spark-website/diff/f57f0702

Branch: refs/heads/asf-site
Commit: f57f0702b293bed20111de50f62eaea5026175a4
Parents: 5ddf243
Author: Michael Armbrust 
Authored: Thu Jul 13 20:17:23 2017 +
Committer: Michael Armbrust 
Committed: Thu Jul 13 20:17:23 2017 +

--
 downloads.md  |  3 +++
 releases/_posts/2017-07-11-spark-release-2-2-0.md |  1 +
 site/downloads.html   |  3 +++
 site/releases/spark-release-2-2-0.html|  2 ++
 site/sitemap.xml  | 12 ++--
 5 files changed, 15 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark-website/blob/f57f0702/downloads.md
--
diff --git a/downloads.md b/downloads.md
index 2f1f50c..bf96c5e 100644
--- a/downloads.md
+++ b/downloads.md
@@ -53,6 +53,9 @@ Spark artifacts are [hosted in Maven 
Central](https://search.maven.org/#search%7
 artifactId: spark-core_2.11
 version: 2.2.0
 
+### Installing with PyPi
+https://pypi.python.org/pypi/pyspark;>PySpark is now available in 
pypi. To install just run `pip install pyspark`.
+
 ### Spark Source Code Management
 If you are interested in working with the newest under-development code or 
contributing to Apache Spark development, you can also check out the master 
branch from Git:
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f57f0702/releases/_posts/2017-07-11-spark-release-2-2-0.md
--
diff --git a/releases/_posts/2017-07-11-spark-release-2-2-0.md 
b/releases/_posts/2017-07-11-spark-release-2-2-0.md
index b630c75..2138eb3 100644
--- a/releases/_posts/2017-07-11-spark-release-2-2-0.md
+++ b/releases/_posts/2017-07-11-spark-release-2-2-0.md
@@ -14,6 +14,7 @@ meta:
 
 Apache Spark 2.2.0 is the third release on the 2.x line. This release removes 
the experimental tag from Structured Streaming. In addition, this release 
focuses more on usability, stability, and polish, resolving over 1100 tickets.
 
+Additionally, we are excited to announce that https://pypi.python.org/pypi/pyspark;>PySpark is now available in 
pypi. To install just run `pip install pyspark`.
 
 To download Apache Spark 2.2.0, visit the downloads page. You can consult JIRA 
for the [detailed 
changes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315420=12338275).
 We have curated a list of high level changes here, grouped by major modules.
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f57f0702/site/downloads.html
--
diff --git a/site/downloads.html b/site/downloads.html
index 35e8920..5d3653d 100644
--- a/site/downloads.html
+++ b/site/downloads.html
@@ -249,6 +249,9 @@ artifactId: spark-core_2.11
 version: 2.2.0
 
 
+Installing with PyPi
+https://pypi.python.org/pypi/pyspark;>PySpark is now available 
in pypi. To install just run pip install pyspark.
+
 Spark Source Code Management
 If you are interested in working with the newest under-development code or 
contributing to Apache Spark development, you can also check out the master 
branch from Git:
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f57f0702/site/releases/spark-release-2-2-0.html
--
diff --git a/site/releases/spark-release-2-2-0.html 
b/site/releases/spark-release-2-2-0.html
index fc43088..56f8d21 100644
--- a/site/releases/spark-release-2-2-0.html
+++ b/site/releases/spark-release-2-2-0.html
@@ -199,6 +199,8 @@
 
 Apache Spark 2.2.0 is the third release on the 2.x line. This release 
removes the experimental tag from Structured Streaming. In addition, this 
release focuses more on usability, stability, and polish, resolving over 1100 
tickets.
 
+Additionally, we are excited to announce that https://pypi.python.org/pypi/pyspark;>PySpark is now available in 
pypi. To install just run pip install pyspark.
+
 To download Apache Spark 2.2.0, visit the downloads page. You can consult JIRA for the https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315420version=12338275;>detailed
 changes. We have curated a list of high level changes here, grouped by 
major modules.
 
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f57f0702/site/sitemap.xml

spark-website git commit: More 2.2.0 Release Notes

2017-07-12 Thread marmbrus
Repository: spark-website
Updated Branches:
  refs/heads/asf-site 40f588bb5 -> 869f8a6fb


More 2.2.0 Release Notes


Project: http://git-wip-us.apache.org/repos/asf/spark-website/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark-website/commit/869f8a6f
Tree: http://git-wip-us.apache.org/repos/asf/spark-website/tree/869f8a6f
Diff: http://git-wip-us.apache.org/repos/asf/spark-website/diff/869f8a6f

Branch: refs/heads/asf-site
Commit: 869f8a6fb1548c773d2d75f63c55005417aeac35
Parents: 40f588b
Author: Michael Armbrust 
Authored: Wed Jul 12 22:54:55 2017 +
Committer: Michael Armbrust 
Committed: Wed Jul 12 22:54:55 2017 +

--
 releases/_posts/2017-07-11-spark-release-2-2-0.md | 4 +++-
 site/releases/spark-release-2-2-0.html| 7 ++-
 2 files changed, 9 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark-website/blob/869f8a6f/releases/_posts/2017-07-11-spark-release-2-2-0.md
--
diff --git a/releases/_posts/2017-07-11-spark-release-2-2-0.md 
b/releases/_posts/2017-07-11-spark-release-2-2-0.md
index 52ae28f..b630c75 100644
--- a/releases/_posts/2017-07-11-spark-release-2-2-0.md
+++ b/releases/_posts/2017-07-11-spark-release-2-2-0.md
@@ -128,6 +128,8 @@ The main focus of SparkR in the 2.2.0 release was adding 
extensive support for e
 
 ### Deprecations
 
+ - **Python**
+   - SPARK-12661: Drop support for Python 2.6
  - **MLlib**
- SPARK-18613: spark.ml LDA classes should not expose spark.mllib in APIs.  
In spark.ml.LDAModel, deprecated `oldLocalModel` and `getModel`.
  - **SparkR**
@@ -143,7 +145,7 @@ The main focus of SparkR in the 2.2.0 release was adding 
extensive support for e
 
 ### Known Issues
 
-- None
+- SPARK-21093: Multiple gapply execution occasionally failed in SparkR
 
 
 ### Credits

http://git-wip-us.apache.org/repos/asf/spark-website/blob/869f8a6f/site/releases/spark-release-2-2-0.html
--
diff --git a/site/releases/spark-release-2-2-0.html 
b/site/releases/spark-release-2-2-0.html
index 61504df..fc43088 100644
--- a/site/releases/spark-release-2-2-0.html
+++ b/site/releases/spark-release-2-2-0.html
@@ -371,6 +371,11 @@
 Deprecations
 
 
+  Python
+
+  SPARK-12661: Drop support for Python 2.6
+
+  
   MLlib
 
   SPARK-18613: spark.ml LDA classes should not expose spark.mllib in 
APIs.  In spark.ml.LDAModel, deprecated oldLocalModel and 
getModel.
@@ -401,7 +406,7 @@
 Known Issues
 
 
-  None
+  SPARK-21093: Multiple gapply execution occasionally failed in SparkR
 
 
 Credits


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



spark-website git commit: Fix 2.2.0 contributor list

2017-07-12 Thread marmbrus
Repository: spark-website
Updated Branches:
  refs/heads/asf-site 2fac17731 -> 40f588bb5


Fix 2.2.0 contributor list


Project: http://git-wip-us.apache.org/repos/asf/spark-website/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark-website/commit/40f588bb
Tree: http://git-wip-us.apache.org/repos/asf/spark-website/tree/40f588bb
Diff: http://git-wip-us.apache.org/repos/asf/spark-website/diff/40f588bb

Branch: refs/heads/asf-site
Commit: 40f588bb525e21e457c5d839937350a5c18172c4
Parents: 2fac177
Author: Michael Armbrust 
Authored: Wed Jul 12 22:46:30 2017 +
Committer: Michael Armbrust 
Committed: Wed Jul 12 15:48:01 2017 -0700

--
 releases/_posts/2017-07-11-spark-release-2-2-0.md | 2 +-
 site/releases/spark-release-2-2-0.html| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark-website/blob/40f588bb/releases/_posts/2017-07-11-spark-release-2-2-0.md
--
diff --git a/releases/_posts/2017-07-11-spark-release-2-2-0.md 
b/releases/_posts/2017-07-11-spark-release-2-2-0.md
index 37d3638..52ae28f 100644
--- a/releases/_posts/2017-07-11-spark-release-2-2-0.md
+++ b/releases/_posts/2017-07-11-spark-release-2-2-0.md
@@ -148,4 +148,4 @@ The main focus of SparkR in the 2.2.0 release was adding 
extensive support for e
 
 ### Credits
 Last but not least, this release would not have been possible without the 
following contributors:
-ALeksander Eskilson, Aaditya Ramesh, Adam Roberts, Adrian Petrescu, Ahmed 
Mahran, Alex Bozarth, Alexander Shorin, Alexander Ulanov, Andrew Duffy, Andrew 
Mills, Andrew Ray, Angus Gerry, Anthony Truchet, Anton Okolnychyi, Artur 
Sukhenko, Bartek Wisniewski, Bijay Pathak, Bill Chambers, Bjarne Fruergaard, 
Brian Cho, Bryan Cutler, Burak Yavuz, Cen Yu Hai, Charles Allen, Cheng Lian, 
Chie Hayashida, Christian Kadner, Clark Fitzgerald, Cody Koeninger, Daniel 
Darabos, Daoyuan Wang, David Navas, Davies Liu, Denny Lee, Devaraj K, Dhruve 
Ashar, Dilip Biswal, Ding Ding, Dmitriy Sokolov, Dongjoon Hyun, Drew Robb, 
Ekasit Kijsipongse, Eren Avsarogullari, Ergin Seyfe, Eric Liang, Erik 
O'Shaughnessy, Eyal Farago, Felix Cheung, Ferdinand Xu, Fred Reiss, Fu Xing, 
Gabriel Huang, Gaetan Semet, Gang Wu, Gayathri Murali, Gu Huiqin Alice, 
Guoqiang Li, Gurvinder Singh, Hao Ren, Herman Van Hovell, Hiroshi Inoue, Holden 
Karau, Hossein Falaki, Huang Zhaowei, Huaxin Gao, Hyukjin Kwon, Imran Rashid, 
Jacek Laskows
 ki, Jagadeesan A S, Jakob Odersky, Jason White, Jeff Zhang, Jianfei Wang, 
Jiang Xingbo, Jie Huang, Jie Xiong, Jisoo Kim, John Muller, Jose Hiram Soltren, 
Joseph K. Bradley, Josh Rosen, Jun Kim, Junyang Qian, Justin Pihony, Kapil 
Singh, Kay Ousterhout, Kazuaki Ishizaki, Kevin Grealish, Kevin McHale, Kishor 
Patil, Koert Kuipers, Kousuke Saruta, Krishna Kalyan, Liang Ke, Liang-Chi 
Hsieh, Lianhui Wang, Linbo Jin, Liwei Lin, Luciano Resende, Maciej Brynski, 
Maciej Szymkiewicz, Mahmoud Rawas, Manoj Kumar, Marcelo Vanzin, Mariusz 
Strzelecki, Mark Grover, Maxime Rihouey, Miao Wang, Michael Allman, Michael 
Armbrust, Michael Gummelt, Michal Senkyr, Michal Wesolowski, Mikael Staldal, 
Mike Ihbe, Mitesh Patel, Nan Zhu, Nattavut Sutyanyong, Nic Eggert, Nicholas 
Chammas, Nick Lavers, Nick Pentreath, Nicolas Fraison, Noritaka Sekiyama, Peng 
Meng, Peng, Meng, Pete Robbins, Peter Ableda, Peter Lee, Philipp Hoffmann, 
Prashant Sharma, Prince J Wesley, Priyanka Garg, Qian Huang, Qifan Pu, Rajesh 
Balamoh
 an, Reynold Xin, Robert Kruszewski, Russell Spitzer, Ryan Blue, Saisai Shao, 
Sameer Agarwal, Sami Jaktholm, Sandeep Purohit, Sandeep Singh, Satendra Kumar, 
Sean Owen, Sean Zhong, Seth Hendrickson, Sharkd Tu, Shen Hong, Shivansh 
Srivastava, Shivaram Venkataraman, Shixiong Zhu, Shuai Lin, Shubham Chopra, 
Sital Kedia, Song Jun, Srinath Shankar, Stavros Kontopoulos, Stefan Schulze, 
Steve Loughran, Suman Somasundar, Sun Dapeng, Sun Rui, Sunitha Kambhampati, 
Suresh Thalamati, Susan X. Huynh, Sylvain Zimmer, Takeshi YAMAMURO, Takuya 
UESHIN, Tao LI, Tao Lin, Tao Wang, Tarun Kumar, Tathagata Das, Tejas Patil, 
Thomas Graves, Timothy Chen, Timothy Hunter, Tom Graves, Tom Magrino, Tommy YU, 
Tyson Condie, Uncle Gen, Vinayak Joshi, Vincent Xie, Wang Fei, Wang Lei, Wang 
Tao, Wayne Zhang, Weichen Xu, Weiluo (David) Ren, Weiqing Yang, Wenchen Fan, 
Wesley Tang, William Benton, Wojciech Szymanski, Xiangrui Meng, Xianyang Liu, 
Xiao Li, Xin Ren, Xin Wu, Xing SHI, Xusen Yin, Yadong Qi, Yanbo Liang, Yang 
 Wang, Yangyang Liu, Yin Huai, Yu Peng, Yucai Yu, Yuhao Yang, Yuming Wang, Yun 
Ni, Yves Raimond, Zhan Zhang, Zheng RuiFeng, Zhenhua Wang, pkch, tone-zhang, 
yimuxi
\ No newline at end of file
+ALeksander Eskilson, Aaditya Ramesh, Adam Budde, Adam Roberts, Adrian Ionescu, 
Ala Luszczak, Alex 

spark-website git commit: Fix downloads page

2017-07-11 Thread marmbrus
Repository: spark-website
Updated Branches:
  refs/heads/asf-site ee654d1f3 -> f2d5d2a68


Fix downloads page


Project: http://git-wip-us.apache.org/repos/asf/spark-website/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark-website/commit/f2d5d2a6
Tree: http://git-wip-us.apache.org/repos/asf/spark-website/tree/f2d5d2a6
Diff: http://git-wip-us.apache.org/repos/asf/spark-website/diff/f2d5d2a6

Branch: refs/heads/asf-site
Commit: f2d5d2a6800e479396a21db2e349db6008df84ec
Parents: ee654d1
Author: Michael Armbrust 
Authored: Tue Jul 11 22:47:12 2017 +
Committer: Michael Armbrust 
Committed: Tue Jul 11 22:47:12 2017 +

--
 downloads.md| 6 +++---
 site/downloads.html | 6 +++---
 2 files changed, 6 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark-website/blob/f2d5d2a6/downloads.md
--
diff --git a/downloads.md b/downloads.md
index 16d6062..2f1f50c 100644
--- a/downloads.md
+++ b/downloads.md
@@ -51,7 +51,7 @@ Spark artifacts are [hosted in Maven 
Central](https://search.maven.org/#search%7
 
 groupId: org.apache.spark
 artifactId: spark-core_2.11
-version: 2.1.1
+version: 2.2.0
 
 ### Spark Source Code Management
 If you are interested in working with the newest under-development code or 
contributing to Apache Spark development, you can also check out the master 
branch from Git:
@@ -59,8 +59,8 @@ If you are interested in working with the newest 
under-development code or contr
 # Master development branch
 git clone git://github.com/apache/spark.git
 
-# 2.1 maintenance branch with stability fixes on top of Spark 2.1.0
-git clone git://github.com/apache/spark.git -b branch-2.1
+# 2.1 maintenance branch with stability fixes on top of Spark 2.2.0
+git clone git://github.com/apache/spark.git -b branch-2.2
 
 Once you've downloaded Spark, you can find instructions for installing and 
building it on the documentation 
page.
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f2d5d2a6/site/downloads.html
--
diff --git a/site/downloads.html b/site/downloads.html
index 93bbcdc..35e8920 100644
--- a/site/downloads.html
+++ b/site/downloads.html
@@ -246,7 +246,7 @@ You can select and download it above.
 
 groupId: org.apache.spark
 artifactId: spark-core_2.11
-version: 2.1.1
+version: 2.2.0
 
 
 Spark Source Code Management
@@ -255,8 +255,8 @@ version: 2.1.1
 # Master development branch
 git clone git://github.com/apache/spark.git
 
-# 2.1 maintenance branch with stability fixes on top of Spark 2.1.0
-git clone git://github.com/apache/spark.git -b branch-2.1
+# 2.1 maintenance branch with stability fixes on top of Spark 2.2.0
+git clone git://github.com/apache/spark.git -b branch-2.2
 
 
 Once youve downloaded Spark, you can find instructions for 
installing and building it on the documentation 
page.


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



[36/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/groupBy.html
--
diff --git a/site/docs/2.2.0/api/R/groupBy.html 
b/site/docs/2.2.0/api/R/groupBy.html
new file mode 100644
index 000..46a23f4
--- /dev/null
+++ b/site/docs/2.2.0/api/R/groupBy.html
@@ -0,0 +1,300 @@
+
+R: GroupBy
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+groupBy 
{SparkR}R Documentation
+
+GroupBy
+
+Description
+
+Groups the SparkDataFrame using the specified columns, so we can run 
aggregation on them.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+groupBy(x, ...)
+
+## S4 method for signature 'SparkDataFrame'
+group_by(x, ...)
+
+group_by(x, ...)
+
+groupBy(x, ...)
+
+
+
+Arguments
+
+
+x
+
+a SparkDataFrame.
+
+...
+
+variable(s) (character names(s) or Column(s)) to group on.
+
+
+
+
+Value
+
+A GroupedData.
+
+
+
+Note
+
+groupBy since 1.4.0
+
+group_by since 1.4.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+intersect, intersect,
+intersect,SparkDataFrame,SparkDataFrame-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,
+join,SparkDataFrame,SparkDataFrame-method;
+limit, limit,
+limit,SparkDataFrame,numeric-method;
+merge, merge,
+merge,SparkDataFrame,SparkDataFrame-method;
+mutate, mutate,

[30/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/nafunctions.html
--
diff --git a/site/docs/2.2.0/api/R/nafunctions.html 
b/site/docs/2.2.0/api/R/nafunctions.html
new file mode 100644
index 000..aaf39cb
--- /dev/null
+++ b/site/docs/2.2.0/api/R/nafunctions.html
@@ -0,0 +1,349 @@
+
+R: A set of SparkDataFrame functions working with NA 
values
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+dropna 
{SparkR}R Documentation
+
+A set of SparkDataFrame functions working with NA values
+
+Description
+
+dropna, na.omit - Returns a new SparkDataFrame omitting rows with null 
values.
+
+fillna - Replace null values.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+dropna(x, how = c("any", "all"),
+  minNonNulls = NULL, cols = NULL)
+
+## S4 method for signature 'SparkDataFrame'
+na.omit(object, how = c("any", "all"),
+  minNonNulls = NULL, cols = NULL)
+
+## S4 method for signature 'SparkDataFrame'
+fillna(x, value, cols = NULL)
+
+dropna(x, how = c("any", "all"), minNonNulls = NULL, cols = NULL)
+
+na.omit(object, ...)
+
+fillna(x, value, cols = NULL)
+
+
+
+Arguments
+
+
+x
+
+a SparkDataFrame.
+
+how
+
+any or all.
+if any, drop a row if it contains any nulls.
+if all, drop a row only if all its values are null.
+if minNonNulls is specified, how is ignored.
+
+minNonNulls
+
+if specified, drop rows that have less than
+minNonNulls non-null values.
+This overwrites the how parameter.
+
+cols
+
+optional list of column names to consider. In fillna,
+columns specified in cols that do not have matching data
+type are ignored. For example, if value is a character, and
+subset contains a non-character column, then the non-character
+column is simply ignored.
+
+object
+
+a SparkDataFrame.
+
+value
+
+value to replace null values with.
+Should be an integer, numeric, character or named list.
+If the value is a named list, then cols is ignored and
+value must be a mapping from column name (character) to
+replacement value. The replacement value must be an
+integer, numeric or character.
+
+...
+
+further arguments to be passed to or from other methods.
+
+
+
+
+Value
+
+A SparkDataFrame.
+
+
+
+Note
+
+dropna since 1.4.0
+
+na.omit since 1.5.0
+
+fillna since 1.4.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,

[24/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/shiftLeft.html
--
diff --git a/site/docs/2.2.0/api/R/shiftLeft.html 
b/site/docs/2.2.0/api/R/shiftLeft.html
new file mode 100644
index 000..eacecc4
--- /dev/null
+++ b/site/docs/2.2.0/api/R/shiftLeft.html
@@ -0,0 +1,126 @@
+
+R: shiftLeft
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+shiftLeft 
{SparkR}R Documentation
+
+shiftLeft
+
+Description
+
+Shift the given value numBits left. If the given value is a long value, 
this function
+will return a long value else it will return an integer value.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column,numeric'
+shiftLeft(y, x)
+
+shiftLeft(y, x)
+
+
+
+Arguments
+
+
+y
+
+column to compute on.
+
+x
+
+number of bits to shift.
+
+
+
+
+Note
+
+shiftLeft since 1.5.0
+
+
+
+See Also
+
+Other math_funcs: acos,
+acos,Column-method; asin,
+asin,Column-method; atan2,
+atan2,Column-method; atan,
+atan,Column-method; bin,
+bin, bin,Column-method;
+bround, bround,
+bround,Column-method; cbrt,
+cbrt, cbrt,Column-method;
+ceil, ceil,
+ceil,Column-method, ceiling,
+ceiling,Column-method; conv,
+conv,
+conv,Column,numeric,numeric-method;
+corr, corr,
+corr, corr,Column-method,
+corr,SparkDataFrame-method;
+cosh, cosh,Column-method;
+cos, cos,Column-method;
+covar_pop, covar_pop,
+covar_pop,characterOrColumn,characterOrColumn-method;
+cov, cov, cov,
+cov,SparkDataFrame-method,
+cov,characterOrColumn-method,
+covar_samp, covar_samp,
+covar_samp,characterOrColumn,characterOrColumn-method;
+expm1, expm1,Column-method;
+exp, exp,Column-method;
+factorial,
+factorial,Column-method;
+floor, floor,Column-method;
+hex, hex,
+hex,Column-method; hypot,
+hypot, hypot,Column-method;
+log10, log10,Column-method;
+log1p, log1p,Column-method;
+log2, log2,Column-method;
+log, log,Column-method;
+pmod, pmod,
+pmod,Column-method; rint,
+rint, rint,Column-method;
+round, round,Column-method;
+shiftRightUnsigned,
+shiftRightUnsigned,
+shiftRightUnsigned,Column,numeric-method;
+shiftRight, shiftRight,
+shiftRight,Column,numeric-method;
+sign, sign,Column-method,
+signum, signum,
+signum,Column-method; sinh,
+sinh,Column-method; sin,
+sin,Column-method; sqrt,
+sqrt,Column-method; tanh,
+tanh,Column-method; tan,
+tan,Column-method; toDegrees,
+toDegrees,
+toDegrees,Column-method;
+toRadians, toRadians,
+toRadians,Column-method;
+unhex, unhex,
+unhex,Column-method
+
+
+
+Examples
+
+## Not run: shiftLeft(df$c, 1)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/shiftRight.html
--
diff --git a/site/docs/2.2.0/api/R/shiftRight.html 
b/site/docs/2.2.0/api/R/shiftRight.html
new file mode 100644
index 000..b7b41b4
--- /dev/null
+++ b/site/docs/2.2.0/api/R/shiftRight.html
@@ -0,0 +1,126 @@
+
+R: shiftRight
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+shiftRight 
{SparkR}R Documentation
+
+shiftRight
+
+Description
+
+(Signed) shift the given value numBits right. If the given value is a long 
value, it will return
+a long value else it will return an integer value.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column,numeric'
+shiftRight(y, x)
+
+shiftRight(y, x)
+
+
+
+Arguments
+
+
+y
+
+column to compute on.
+
+x
+
+number of bits to shift.
+
+
+
+
+Note
+
+shiftRight since 1.5.0
+
+
+
+See Also
+
+Other math_funcs: acos,
+acos,Column-method; asin,
+asin,Column-method; atan2,
+atan2,Column-method; atan,
+atan,Column-method; bin,
+bin, bin,Column-method;
+bround, bround,
+bround,Column-method; cbrt,
+cbrt, cbrt,Column-method;
+ceil, ceil,
+ceil,Column-method, ceiling,
+ceiling,Column-method; conv,
+conv,
+conv,Column,numeric,numeric-method;
+corr, corr,
+corr, corr,Column-method,
+corr,SparkDataFrame-method;
+cosh, cosh,Column-method;
+cos, cos,Column-method;
+covar_pop, covar_pop,
+covar_pop,characterOrColumn,characterOrColumn-method;
+cov, cov, cov,
+cov,SparkDataFrame-method,
+cov,characterOrColumn-method,
+covar_samp, covar_samp,
+covar_samp,characterOrColumn,characterOrColumn-method;
+expm1, expm1,Column-method;
+exp, exp,Column-method;
+factorial,
+factorial,Column-method;
+floor, floor,Column-method;
+hex, hex,
+hex,Column-method; hypot,
+hypot, hypot,Column-method;
+log10, log10,Column-method;
+log1p, log1p,Column-method;
+log2, log2,Column-method;
+log, log,Column-method;
+pmod, pmod,

[32/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/listColumns.html
--
diff --git a/site/docs/2.2.0/api/R/listColumns.html 
b/site/docs/2.2.0/api/R/listColumns.html
new file mode 100644
index 000..0c486a7
--- /dev/null
+++ b/site/docs/2.2.0/api/R/listColumns.html
@@ -0,0 +1,67 @@
+
+R: Returns a list of columns for the given table/view in 
the...
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+listColumns {SparkR}R 
Documentation
+
+Returns a list of columns for the given table/view in the specified 
database
+
+Description
+
+Returns a list of columns for the given table/view in the specified 
database.
+
+
+
+Usage
+
+
+listColumns(tableName, databaseName = NULL)
+
+
+
+Arguments
+
+
+tableName
+
+the qualified or unqualified name that designates a table/view. If no 
database
+identifier is provided, it refers to a table/view in the current database.
+If databaseName parameter is specified, this must be an 
unqualified name.
+
+databaseName
+
+(optional) name of the database
+
+
+
+
+Value
+
+a SparkDataFrame of the list of column descriptions.
+
+
+
+Note
+
+since 2.2.0
+
+
+
+Examples
+
+## Not run: 
+##D sparkR.session()
+##D listColumns(mytable)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/listDatabases.html
--
diff --git a/site/docs/2.2.0/api/R/listDatabases.html 
b/site/docs/2.2.0/api/R/listDatabases.html
new file mode 100644
index 000..e903961
--- /dev/null
+++ b/site/docs/2.2.0/api/R/listDatabases.html
@@ -0,0 +1,51 @@
+
+R: Returns a list of databases available
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+listDatabases {SparkR}R 
Documentation
+
+Returns a list of databases available
+
+Description
+
+Returns a list of databases available.
+
+
+
+Usage
+
+
+listDatabases()
+
+
+
+Value
+
+a SparkDataFrame of the list of databases.
+
+
+
+Note
+
+since 2.2.0
+
+
+
+Examples
+
+## Not run: 
+##D sparkR.session()
+##D listDatabases()
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/listFunctions.html
--
diff --git a/site/docs/2.2.0/api/R/listFunctions.html 
b/site/docs/2.2.0/api/R/listFunctions.html
new file mode 100644
index 000..85ae31b
--- /dev/null
+++ b/site/docs/2.2.0/api/R/listFunctions.html
@@ -0,0 +1,62 @@
+
+R: Returns a list of functions registered in the 
specified...
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+listFunctions {SparkR}R 
Documentation
+
+Returns a list of functions registered in the specified database
+
+Description
+
+Returns a list of functions registered in the specified database.
+This includes all temporary functions.
+
+
+
+Usage
+
+
+listFunctions(databaseName = NULL)
+
+
+
+Arguments
+
+
+databaseName
+
+(optional) name of the database
+
+
+
+
+Value
+
+a SparkDataFrame of the list of function descriptions.
+
+
+
+Note
+
+since 2.2.0
+
+
+
+Examples
+
+## Not run: 
+##D sparkR.session()
+##D listFunctions()
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/listTables.html
--
diff --git a/site/docs/2.2.0/api/R/listTables.html 
b/site/docs/2.2.0/api/R/listTables.html
new file mode 100644
index 000..a418a55
--- /dev/null
+++ b/site/docs/2.2.0/api/R/listTables.html
@@ -0,0 +1,69 @@
+
+R: Returns a list of tables or views in the specified 
database
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+listTables 
{SparkR}R Documentation
+
+Returns a list of tables or views in the specified database
+
+Description
+
+Returns a list of tables or views in the specified 

[42/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/crosstab.html
--
diff --git a/site/docs/2.2.0/api/R/crosstab.html 
b/site/docs/2.2.0/api/R/crosstab.html
new file mode 100644
index 000..e944279
--- /dev/null
+++ b/site/docs/2.2.0/api/R/crosstab.html
@@ -0,0 +1,94 @@
+
+R: Computes a pair-wise frequency table of the given 
columns
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+crosstab 
{SparkR}R Documentation
+
+Computes a pair-wise frequency table of the given columns
+
+Description
+
+Computes a pair-wise frequency table of the given columns. Also known as a 
contingency
+table. The number of distinct values for each column should be less than 1e4. 
At most 1e6
+non-zero pair frequencies will be returned.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,character,character'
+crosstab(x, col1, col2)
+
+
+
+Arguments
+
+
+x
+
+a SparkDataFrame
+
+col1
+
+name of the first column. Distinct items will make the first item of each 
row.
+
+col2
+
+name of the second column. Distinct items will make the column names of the 
output.
+
+
+
+
+Value
+
+a local R data.frame representing the contingency table. The first column 
of each row
+will be the distinct values of col1 and the column names will be 
the distinct values
+of col2. The name of the first column will be 
col1_col2. Pairs
+that have no occurrences will have zero as their counts.
+
+
+
+Note
+
+crosstab since 1.5.0
+
+
+
+See Also
+
+Other stat functions: approxQuantile,
+approxQuantile,SparkDataFrame,character,numeric,numeric-method;
+corr, corr,
+corr, corr,Column-method,
+corr,SparkDataFrame-method;
+cov, cov, cov,
+cov,SparkDataFrame-method,
+cov,characterOrColumn-method,
+covar_samp, covar_samp,
+covar_samp,characterOrColumn,characterOrColumn-method;
+freqItems,
+freqItems,SparkDataFrame,character-method;
+sampleBy, sampleBy,
+sampleBy,SparkDataFrame,character,list,numeric-method
+
+
+
+Examples
+
+## Not run: 
+##D df - read.json(/path/to/file.json)
+##D ct - crosstab(df, title, gender)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/cume_dist.html
--
diff --git a/site/docs/2.2.0/api/R/cume_dist.html 
b/site/docs/2.2.0/api/R/cume_dist.html
new file mode 100644
index 000..328243d
--- /dev/null
+++ b/site/docs/2.2.0/api/R/cume_dist.html
@@ -0,0 +1,90 @@
+
+R: cume_dist
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+cume_dist 
{SparkR}R Documentation
+
+cume_dist
+
+Description
+
+Window function: returns the cumulative distribution of values within a 
window partition,
+i.e. the fraction of rows that are below the current row.
+
+
+
+Usage
+
+
+## S4 method for signature 'missing'
+cume_dist()
+
+cume_dist(x = "missing")
+
+
+
+Arguments
+
+
+x
+
+empty. Should be used with no argument.
+
+
+
+
+Details
+
+N = total number of rows in the partition
+cume_dist(x) = number of values before (and including) x / N
+
+This is equivalent to the CUME_DIST function in SQL.
+
+
+
+Note
+
+cume_dist since 1.6.0
+
+
+
+See Also
+
+Other window_funcs: dense_rank,
+dense_rank,
+dense_rank,missing-method;
+lag, lag,
+lag,characterOrColumn-method;
+lead, lead,
+lead,characterOrColumn,numeric-method;
+ntile, ntile,
+ntile,numeric-method;
+percent_rank, percent_rank,
+percent_rank,missing-method;
+rank, rank,
+rank, rank,ANY-method,
+rank,missing-method;
+row_number, row_number,
+row_number,missing-method
+
+
+
+Examples
+
+## Not run: 
+##D   df - createDataFrame(mtcars)
+##D   ws - orderBy(windowPartitionBy(am), hp)
+##D   out - select(df, over(cume_dist(), ws), df$hp, df$am)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/currentDatabase.html
--
diff --git a/site/docs/2.2.0/api/R/currentDatabase.html 
b/site/docs/2.2.0/api/R/currentDatabase.html
new file mode 100644
index 000..0b9d463
--- /dev/null
+++ b/site/docs/2.2.0/api/R/currentDatabase.html
@@ -0,0 +1,51 @@
+
+R: Returns the current default database
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>

[11/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/constant-values.html
--
diff --git a/site/docs/2.2.0/api/java/constant-values.html 
b/site/docs/2.2.0/api/java/constant-values.html
new file mode 100644
index 000..7a7e0d6
--- /dev/null
+++ b/site/docs/2.2.0/api/java/constant-values.html
@@ -0,0 +1,242 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+Constant Field Values (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+Prev
+Next
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+
+
+
+Constant Field Values
+Contents
+
+org.apache.*
+
+
+
+
+
+org.apache.*
+
+
+
+org.apache.spark.launcher.SparkLauncher
+
+Modifier and Type
+Constant Field
+Value
+
+
+
+
+
+publicstaticfinalString
+CHILD_CONNECTION_TIMEOUT
+"spark.launcher.childConectionTimeout"
+
+
+
+
+publicstaticfinalString
+CHILD_PROCESS_LOGGER_NAME
+"spark.launcher.childProcLoggerName"
+
+
+
+
+publicstaticfinalString
+DEPLOY_MODE
+"spark.submit.deployMode"
+
+
+
+
+publicstaticfinalString
+DRIVER_EXTRA_CLASSPATH
+"spark.driver.extraClassPath"
+
+
+
+
+publicstaticfinalString
+DRIVER_EXTRA_JAVA_OPTIONS
+"spark.driver.extraJavaOptions"
+
+
+
+
+publicstaticfinalString
+DRIVER_EXTRA_LIBRARY_PATH
+"spark.driver.extraLibraryPath"
+
+
+
+
+publicstaticfinalString
+DRIVER_MEMORY
+"spark.driver.memory"
+
+
+
+
+publicstaticfinalString
+EXECUTOR_CORES
+"spark.executor.cores"
+
+
+
+
+publicstaticfinalString
+EXECUTOR_EXTRA_CLASSPATH
+"spark.executor.extraClassPath"
+
+
+
+
+publicstaticfinalString
+EXECUTOR_EXTRA_JAVA_OPTIONS
+"spark.executor.extraJavaOptions"
+
+
+
+
+publicstaticfinalString
+EXECUTOR_EXTRA_LIBRARY_PATH
+"spark.executor.extraLibraryPath"
+
+
+
+
+publicstaticfinalString
+EXECUTOR_MEMORY
+"spark.executor.memory"
+
+
+
+
+publicstaticfinalString
+NO_RESOURCE
+"spark-internal"
+
+
+
+
+publicstaticfinalString
+SPARK_MASTER
+"spark.master"
+
+
+
+
+
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+Prev
+Next
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/deprecated-list.html
--
diff --git a/site/docs/2.2.0/api/java/deprecated-list.html 
b/site/docs/2.2.0/api/java/deprecated-list.html
new file mode 100644
index 000..2e18a65
--- /dev/null
+++ b/site/docs/2.2.0/api/java/deprecated-list.html
@@ -0,0 +1,841 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+Deprecated List (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+Prev
+Next
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+
+
+
+Deprecated API
+Contents
+
+Deprecated Interfaces
+Deprecated Classes
+Deprecated Methods
+Deprecated Constructors
+
+
+
+
+
+
+
+
+Deprecated Interfaces
+
+Interface and Description
+
+
+
+org.apache.spark.AccumulableParam
+use AccumulatorV2. Since 
2.0.0.
+
+
+
+org.apache.spark.AccumulatorParam
+use AccumulatorV2. Since 
2.0.0.
+
+
+
+
+
+
+
+
+
+
+
+
+Deprecated Classes
+
+Class and Description
+
+
+
+org.apache.spark.Accumulable
+use AccumulatorV2. Since 
2.0.0.
+
+
+
+org.apache.spark.Accumulator
+use AccumulatorV2. Since 
2.0.0.
+
+
+
+org.apache.spark.AccumulatorParam.DoubleAccumulatorParam$
+use AccumulatorV2. Since 
2.0.0.
+
+
+
+org.apache.spark.AccumulatorParam.FloatAccumulatorParam$
+use AccumulatorV2. Since 
2.0.0.
+
+
+
+org.apache.spark.AccumulatorParam.IntAccumulatorParam$
+use AccumulatorV2. Since 
2.0.0.
+
+
+
+org.apache.spark.AccumulatorParam.LongAccumulatorParam$
+use AccumulatorV2. 

[19/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/tableNames.html
--
diff --git a/site/docs/2.2.0/api/R/tableNames.html 
b/site/docs/2.2.0/api/R/tableNames.html
new file mode 100644
index 000..3d0c8ea
--- /dev/null
+++ b/site/docs/2.2.0/api/R/tableNames.html
@@ -0,0 +1,62 @@
+
+R: Table Names
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+tableNames 
{SparkR}R Documentation
+
+Table Names
+
+Description
+
+Returns the names of tables in the given database as an array.
+
+
+
+Usage
+
+
+## Default S3 method:
+tableNames(databaseName = NULL)
+
+
+
+Arguments
+
+
+databaseName
+
+(optional) name of the database
+
+
+
+
+Value
+
+a list of table names
+
+
+
+Note
+
+tableNames since 1.4.0
+
+
+
+Examples
+
+## Not run: 
+##D sparkR.session()
+##D tableNames(hive)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/tableToDF.html
--
diff --git a/site/docs/2.2.0/api/R/tableToDF.html 
b/site/docs/2.2.0/api/R/tableToDF.html
new file mode 100644
index 000..086636f
--- /dev/null
+++ b/site/docs/2.2.0/api/R/tableToDF.html
@@ -0,0 +1,68 @@
+
+R: Create a SparkDataFrame from a SparkSQL table or 
view
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+tableToDF 
{SparkR}R Documentation
+
+Create a SparkDataFrame from a SparkSQL table or view
+
+Description
+
+Returns the specified table or view as a SparkDataFrame. The table or view 
must already exist or
+have already been registered in the SparkSession.
+
+
+
+Usage
+
+
+tableToDF(tableName)
+
+
+
+Arguments
+
+
+tableName
+
+the qualified or unqualified name that designates a table or view. If a 
database
+is specified, it identifies the table/view from the database.
+Otherwise, it first attempts to find a temporary view with the given name
+and then match the table/view from the current database.
+
+
+
+
+Value
+
+SparkDataFrame
+
+
+
+Note
+
+tableToDF since 2.0.0
+
+
+
+Examples
+
+## Not run: 
+##D sparkR.session()
+##D path - path/to/file.json
+##D df - read.json(path)
+##D createOrReplaceTempView(df, table)
+##D new_df - tableToDF(table)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/tables.html
--
diff --git a/site/docs/2.2.0/api/R/tables.html 
b/site/docs/2.2.0/api/R/tables.html
new file mode 100644
index 000..555027e
--- /dev/null
+++ b/site/docs/2.2.0/api/R/tables.html
@@ -0,0 +1,68 @@
+
+R: Tables
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+tables 
{SparkR}R Documentation
+
+Tables
+
+Description
+
+Returns a SparkDataFrame containing names of tables in the given database.
+
+
+
+Usage
+
+
+## Default S3 method:
+tables(databaseName = NULL)
+
+
+
+Arguments
+
+
+databaseName
+
+(optional) name of the database
+
+
+
+
+Value
+
+a SparkDataFrame
+
+
+
+Note
+
+tables since 1.4.0
+
+
+
+See Also
+
+listTables
+
+
+
+Examples
+
+## Not run: 
+##D sparkR.session()
+##D tables(hive)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/take.html
--
diff --git a/site/docs/2.2.0/api/R/take.html b/site/docs/2.2.0/api/R/take.html
new file mode 100644
index 000..e9af127
--- /dev/null
+++ b/site/docs/2.2.0/api/R/take.html
@@ -0,0 +1,288 @@
+
+R: Take the first NUM rows of a SparkDataFrame and return 
the...
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+take 
{SparkR}R Documentation
+
+Take the first NUM rows of a SparkDataFrame and return the results as a R 
data.frame
+
+Description
+
+Take the first NUM rows of a SparkDataFrame and return the results as a R 

[22/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/spark.lda.html
--
diff --git a/site/docs/2.2.0/api/R/spark.lda.html 
b/site/docs/2.2.0/api/R/spark.lda.html
new file mode 100644
index 000..aa1cfa8
--- /dev/null
+++ b/site/docs/2.2.0/api/R/spark.lda.html
@@ -0,0 +1,247 @@
+
+R: Latent Dirichlet Allocation
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+spark.lda 
{SparkR}R Documentation
+
+Latent Dirichlet Allocation
+
+Description
+
+spark.lda fits a Latent Dirichlet Allocation model on a 
SparkDataFrame. Users can call
+summary to get a summary of the fitted LDA model, 
spark.posterior to compute
+posterior probabilities on new data, spark.perplexity to compute 
log perplexity on new
+data and write.ml/read.ml to save/load fitted models.
+
+
+
+Usage
+
+
+spark.lda(data, ...)
+
+spark.posterior(object, newData)
+
+spark.perplexity(object, data)
+
+## S4 method for signature 'SparkDataFrame'
+spark.lda(data, features = "features", k = 10,
+  maxIter = 20, optimizer = c("online", "em"), subsamplingRate = 0.05,
+  topicConcentration = -1, docConcentration = -1,
+  customizedStopWords = "", maxVocabSize = bitwShiftL(1, 18))
+
+## S4 method for signature 'LDAModel'
+summary(object, maxTermsPerTopic)
+
+## S4 method for signature 'LDAModel,SparkDataFrame'
+spark.perplexity(object, data)
+
+## S4 method for signature 'LDAModel,SparkDataFrame'
+spark.posterior(object, newData)
+
+## S4 method for signature 'LDAModel,character'
+write.ml(object, path, overwrite = FALSE)
+
+
+
+Arguments
+
+
+data
+
+A SparkDataFrame for training.
+
+...
+
+additional argument(s) passed to the method.
+
+object
+
+A Latent Dirichlet Allocation model fitted by spark.lda.
+
+newData
+
+A SparkDataFrame for testing.
+
+features
+
+Features column name. Either libSVM-format column or character-format 
column is
+valid.
+
+k
+
+Number of topics.
+
+maxIter
+
+Maximum iterations.
+
+optimizer
+
+Optimizer to train an LDA model, online or em, 
default is online.
+
+subsamplingRate
+
+(For online optimizer) Fraction of the corpus to be sampled and used in
+each iteration of mini-batch gradient descent, in range (0, 1].
+
+topicConcentration
+
+concentration parameter (commonly named beta or 
eta) for
+the prior placed on topic distributions over terms, default -1 to set 
automatically on the
+Spark side. Use summary to retrieve the effective 
topicConcentration. Only 1-size
+numeric is accepted.
+
+docConcentration
+
+concentration parameter (commonly named alpha) for the
+prior placed on documents distributions over topics (theta), 
default -1 to set
+automatically on the Spark side. Use summary to retrieve the 
effective
+docConcentration. Only 1-size or k-size numeric is accepted.
+
+customizedStopWords
+
+stopwords that need to be removed from the given corpus. Ignore the
+parameter if libSVM-format column is used as the features column.
+
+maxVocabSize
+
+maximum vocabulary size, default 1  18
+
+maxTermsPerTopic
+
+Maximum number of terms to collect for each topic. Default value of 10.
+
+path
+
+The directory where the model is saved.
+
+overwrite
+
+Overwrites or not if the output path already exists. Default is FALSE
+which means throw exception if the output path exists.
+
+
+
+
+Value
+
+spark.lda returns a fitted Latent Dirichlet Allocation model.
+
+summary returns summary information of the fitted model, which 
is a list.
+The list includes
+
+
+docConcentration
+
+concentration parameter commonly named alpha for
+the prior placed on documents distributions over topics theta
+
+topicConcentration
+
+concentration parameter commonly named beta or
+eta for the prior placed on topic distributions over terms
+
+logLikelihood
+
+log likelihood of the entire corpus
+
+logPerplexity
+
+log perplexity
+
+isDistributed
+
+TRUE for distributed model while FALSE for local model
+
+vocabSize
+
+number of terms in the corpus
+
+topics
+
+top 10 terms and their weights of all topics
+
+vocabulary
+
+whole terms of the training corpus, NULL if libsvm format file
+used as training set
+
+trainingLogLikelihood
+
+Log likelihood of the observed tokens in the training set,
+given the current parameter estimates:
+log P(docs | topics, topic distributions for docs, Dirichlet hyperparameters)
+It is only for distributed LDA model (i.e., optimizer = em)
+
+logPrior
+
+Log probability of the current parameter estimate:
+log P(topics, topic distributions for docs | Dirichlet hyperparameters)
+It is only for distributed LDA model (i.e., optimizer = em)
+
+
+spark.perplexity returns the log perplexity of given 
SparkDataFrame, or the log
+perplexity of the training data if missing argument data.
+

[34/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/intersect.html
--
diff --git a/site/docs/2.2.0/api/R/intersect.html 
b/site/docs/2.2.0/api/R/intersect.html
new file mode 100644
index 000..c11dcb2
--- /dev/null
+++ b/site/docs/2.2.0/api/R/intersect.html
@@ -0,0 +1,295 @@
+
+R: Intersect
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+intersect 
{SparkR}R Documentation
+
+Intersect
+
+Description
+
+Return a new SparkDataFrame containing rows only in both this SparkDataFrame
+and another SparkDataFrame. This is equivalent to INTERSECT in 
SQL.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,SparkDataFrame'
+intersect(x, y)
+
+intersect(x, y)
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame
+
+y
+
+A SparkDataFrame
+
+
+
+
+Value
+
+A SparkDataFrame containing the result of the intersect.
+
+
+
+Note
+
+intersect since 1.4.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,
+join,SparkDataFrame,SparkDataFrame-method;
+limit, limit,
+limit,SparkDataFrame,numeric-method;
+merge, merge,
+merge,SparkDataFrame,SparkDataFrame-method;
+mutate, mutate,

[49/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/00Index.html
--
diff --git a/site/docs/2.2.0/api/R/00Index.html 
b/site/docs/2.2.0/api/R/00Index.html
new file mode 100644
index 000..7b199db
--- /dev/null
+++ b/site/docs/2.2.0/api/R/00Index.html
@@ -0,0 +1,1714 @@
+
+R: R Frontend for Apache Spark
+
+
+
+ R Frontend for Apache Spark
+http://stat.ethz.ch/R-manual/R-devel/doc/html/logo.jpg; alt="[R logo]">
+
+
+
+http://stat.ethz.ch/R-manual/R-devel/doc/html/packages.html;>http://stat.ethz.ch/R-manual/R-devel/doc/html/left.jpg; alt="[Up]" 
width="30" height="30" border="0">
+http://stat.ethz.ch/R-manual/R-devel/doc/html/index.html;>http://stat.ethz.ch/R-manual/R-devel/doc/html/up.jpg; alt="[Top]" 
width="30" height="30" border="0">
+Documentation for package SparkR version 2.2.0
+
+DESCRIPTION file.
+
+
+Help Pages
+
+
+
+A
+B
+C
+D
+E
+F
+G
+H
+I
+J
+K
+L
+M
+N
+O
+P
+Q
+R
+S
+T
+U
+V
+W
+Y
+misc
+
+
+
+-- A --
+
+
+abs
+abs
+abs-method
+abs
+acos
+acos
+acos-method
+acos
+add_months
+add_months
+add_months-method
+add_months
+AFTSurvivalRegressionModel-class
+S4 class that represents a AFTSurvivalRegressionModel
+agg
+Summarize data across columns
+agg-method
+Summarize data across columns
+alias
+alias
+alias-method
+alias
+ALSModel-class
+S4 class that represents an ALSModel
+approxCountDistinct
+Returns the approximate number of distinct items in a group
+approxCountDistinct-method
+Returns the approximate number of distinct items in a group
+approxQuantile
+Calculates the approximate quantiles of numerical columns of a 
SparkDataFrame
+approxQuantile-method
+Calculates the approximate quantiles of numerical columns of a 
SparkDataFrame
+arrange
+Arrange Rows by Variables
+arrange-method
+Arrange Rows by Variables
+array_contains
+array_contains
+array_contains-method
+array_contains
+as.data.frame
+Download data from a SparkDataFrame into a R data.frame
+as.data.frame-method
+Download data from a SparkDataFrame into a R data.frame
+as.DataFrame
+Create a SparkDataFrame
+as.DataFrame.default
+Create a SparkDataFrame
+asc
+A set of operations working with SparkDataFrame columns
+ascii
+ascii
+ascii-method
+ascii
+asin
+asin
+asin-method
+asin
+associationRules-method
+FP-growth
+atan
+atan
+atan-method
+atan
+atan2
+atan2
+atan2-method
+atan2
+attach
+Attach SparkDataFrame to R search path
+attach-method
+Attach SparkDataFrame to R search path
+avg
+avg
+avg-method
+avg
+awaitTermination
+awaitTermination
+awaitTermination-method
+awaitTermination
+
+
+-- B --
+
+
+base64
+base64
+base64-method
+base64
+between
+between
+between-method
+between
+bin
+bin
+bin-method
+bin
+BisectingKMeansModel-class
+S4 class that represents a BisectingKMeansModel
+bitwiseNOT
+bitwiseNOT
+bitwiseNOT-method
+bitwiseNOT
+bround
+bround
+bround-method
+bround
+
+
+-- C --
+
+
+cache
+Cache
+cache-method
+Cache
+cacheTable
+Cache Table
+cacheTable.default
+Cache Table
+cancelJobGroup
+Cancel active jobs for the specified group
+cancelJobGroup.default
+Cancel active jobs for the specified group
+cast
+Casts the column to a different data type.
+cast-method
+Casts the column to a different data type.
+cbrt
+cbrt
+cbrt-method
+cbrt
+ceil
+Computes the ceiling of the given value
+ceil-method
+Computes the ceiling of the given value
+ceiling
+Computes the ceiling of the given value
+ceiling-method
+Computes the ceiling of the given value
+checkpoint
+checkpoint
+checkpoint-method
+checkpoint
+clearCache
+Clear Cache
+clearCache.default
+Clear Cache
+clearJobGroup
+Clear current job group ID and its description
+clearJobGroup.default
+Clear current job group ID and its description
+coalesce
+Coalesce
+coalesce-method
+Coalesce
+collect
+Collects all the elements of a SparkDataFrame and coerces them into an R 
data.frame.
+collect-method
+Collects all the elements of a SparkDataFrame and coerces them into an R 
data.frame.
+colnames
+Column Names of SparkDataFrame
+colnames-method
+Column Names of SparkDataFrame
+colnames-
+Column Names of SparkDataFrame
+colnames--method
+Column Names of SparkDataFrame
+coltypes
+coltypes
+coltypes-method
+coltypes
+coltypes-
+coltypes
+coltypes--method
+coltypes
+column
+S4 class that represents a SparkDataFrame column
+Column-class
+S4 class that represents a SparkDataFrame column
+column-method
+S4 class that represents a SparkDataFrame column
+columnfunctions
+A set of operations working with SparkDataFrame columns
+columns
+Column Names of SparkDataFrame
+columns-method
+Column Names of SparkDataFrame
+concat
+concat
+concat-method
+concat
+concat_ws
+concat_ws
+concat_ws-method
+concat_ws
+contains
+A set of operations working with SparkDataFrame columns
+conv
+conv
+conv-method
+conv
+corr
+corr
+corr-method
+corr
+cos
+cos
+cos-method
+cos
+cosh
+cosh
+cosh-method
+cosh
+count
+Returns the number of items in a group
+count-method
+Returns the number of items in a group

[41/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/dayofyear.html
--
diff --git a/site/docs/2.2.0/api/R/dayofyear.html 
b/site/docs/2.2.0/api/R/dayofyear.html
new file mode 100644
index 000..a6da86e
--- /dev/null
+++ b/site/docs/2.2.0/api/R/dayofyear.html
@@ -0,0 +1,118 @@
+
+R: dayofyear
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+dayofyear 
{SparkR}R Documentation
+
+dayofyear
+
+Description
+
+Extracts the day of the year as an integer from a given 
date/timestamp/string.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+dayofyear(x)
+
+dayofyear(x)
+
+
+
+Arguments
+
+
+x
+
+Column to compute on.
+
+
+
+
+Note
+
+dayofyear since 1.5.0
+
+
+
+See Also
+
+Other datetime_funcs: add_months,
+add_months,
+add_months,Column,numeric-method;
+date_add, date_add,
+date_add,Column,numeric-method;
+date_format, date_format,
+date_format,Column,character-method;
+date_sub, date_sub,
+date_sub,Column,numeric-method;
+datediff, datediff,
+datediff,Column-method;
+dayofmonth, dayofmonth,
+dayofmonth,Column-method;
+from_unixtime, from_unixtime,
+from_unixtime,Column-method;
+from_utc_timestamp,
+from_utc_timestamp,
+from_utc_timestamp,Column,character-method;
+hour, hour,
+hour,Column-method; last_day,
+last_day,
+last_day,Column-method;
+minute, minute,
+minute,Column-method;
+months_between,
+months_between,
+months_between,Column-method;
+month, month,
+month,Column-method;
+next_day, next_day,
+next_day,Column,character-method;
+quarter, quarter,
+quarter,Column-method;
+second, second,
+second,Column-method;
+to_date, to_date,
+to_date,
+to_date,Column,character-method,
+to_date,Column,missing-method;
+to_timestamp, to_timestamp,
+to_timestamp,
+to_timestamp,Column,character-method,
+to_timestamp,Column,missing-method;
+to_utc_timestamp,
+to_utc_timestamp,
+to_utc_timestamp,Column,character-method;
+unix_timestamp,
+unix_timestamp,
+unix_timestamp,
+unix_timestamp,
+unix_timestamp,Column,character-method,
+unix_timestamp,Column,missing-method,
+unix_timestamp,missing,missing-method;
+weekofyear, weekofyear,
+weekofyear,Column-method;
+window, window,
+window,Column-method; year,
+year, year,Column-method
+
+
+
+Examples
+
+## Not run: dayofyear(df$c)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/decode.html
--
diff --git a/site/docs/2.2.0/api/R/decode.html 
b/site/docs/2.2.0/api/R/decode.html
new file mode 100644
index 000..f5500b1
--- /dev/null
+++ b/site/docs/2.2.0/api/R/decode.html
@@ -0,0 +1,119 @@
+
+R: decode
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+decode 
{SparkR}R Documentation
+
+decode
+
+Description
+
+Computes the first argument into a string from a binary using the provided 
character set
+(one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16').
+
+
+
+Usage
+
+
+## S4 method for signature 'Column,character'
+decode(x, charset)
+
+decode(x, charset)
+
+
+
+Arguments
+
+
+x
+
+Column to compute on.
+
+charset
+
+Character set to use
+
+
+
+
+Note
+
+decode since 1.6.0
+
+
+
+See Also
+
+Other string_funcs: ascii,
+ascii, ascii,Column-method;
+base64, base64,
+base64,Column-method;
+concat_ws, concat_ws,
+concat_ws,character,Column-method;
+concat, concat,
+concat,Column-method; encode,
+encode,
+encode,Column,character-method;
+format_number, format_number,
+format_number,Column,numeric-method;
+format_string, format_string,
+format_string,character,Column-method;
+initcap, initcap,
+initcap,Column-method; instr,
+instr,
+instr,Column,character-method;
+length, length,Column-method;
+levenshtein, levenshtein,
+levenshtein,Column-method;
+locate, locate,
+locate,character,Column-method;
+lower, lower,
+lower,Column-method; lpad,
+lpad,
+lpad,Column,numeric,character-method;
+ltrim, ltrim,
+ltrim,Column-method;
+regexp_extract,
+regexp_extract,
+regexp_extract,Column,character,numeric-method;
+regexp_replace,
+regexp_replace,
+regexp_replace,Column,character,character-method;
+reverse, reverse,
+reverse,Column-method; rpad,
+rpad,
+rpad,Column,numeric,character-method;
+rtrim, rtrim,
+rtrim,Column-method; soundex,
+soundex,
+soundex,Column-method;
+substring_index,
+substring_index,
+substring_index,Column,character,numeric-method;
+translate, translate,
+translate,Column,character,character-method;
+trim, 

[38/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/first.html
--
diff --git a/site/docs/2.2.0/api/R/first.html b/site/docs/2.2.0/api/R/first.html
new file mode 100644
index 000..2a4edc9
--- /dev/null
+++ b/site/docs/2.2.0/api/R/first.html
@@ -0,0 +1,353 @@
+
+R: Return the first row of a SparkDataFrame
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+first 
{SparkR}R Documentation
+
+Return the first row of a SparkDataFrame
+
+Description
+
+Return the first row of a SparkDataFrame
+
+Aggregate function: returns the first value in a group.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+first(x)
+
+## S4 method for signature 'characterOrColumn'
+first(x, na.rm = FALSE)
+
+first(x, ...)
+
+
+
+Arguments
+
+
+x
+
+a SparkDataFrame or a column used in aggregation function.
+
+na.rm
+
+a logical value indicating whether NA values should be stripped
+before the computation proceeds.
+
+...
+
+further arguments to be passed to or from other methods.
+
+
+
+
+Details
+
+The function by default returns the first values it sees. It will return 
the first non-missing
+value it sees when na.rm is set to true. If all values are missing, then NA is 
returned.
+
+
+
+Note
+
+first(SparkDataFrame) since 1.4.0
+
+first(characterOrColumn) since 1.4.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,

[37/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/gapply.html
--
diff --git a/site/docs/2.2.0/api/R/gapply.html 
b/site/docs/2.2.0/api/R/gapply.html
new file mode 100644
index 000..2c46519
--- /dev/null
+++ b/site/docs/2.2.0/api/R/gapply.html
@@ -0,0 +1,374 @@
+
+R: gapply
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+gapply 
{SparkR}R Documentation
+
+gapply
+
+Description
+
+Groups the SparkDataFrame using the specified columns and applies the R 
function to each
+group.
+
+gapply
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+gapply(x, cols, func, schema)
+
+gapply(x, ...)
+
+## S4 method for signature 'GroupedData'
+gapply(x, func, schema)
+
+
+
+Arguments
+
+
+x
+
+a SparkDataFrame or GroupedData.
+
+cols
+
+grouping columns.
+
+func
+
+a function to be applied to each group partition specified by grouping
+column of the SparkDataFrame. The function func takes as argument
+a key - grouping columns and a data frame - a local R data.frame.
+The output of func is a local R data.frame.
+
+schema
+
+the schema of the resulting SparkDataFrame after the function is applied.
+The schema must match to output of func. It has to be defined for 
each
+output column with preferred output column name and corresponding data 
type.
+
+...
+
+additional argument(s) passed to the method.
+
+
+
+
+Value
+
+A SparkDataFrame.
+
+
+
+Note
+
+gapply(SparkDataFrame) since 2.0.0
+
+gapply(GroupedData) since 2.0.0
+
+
+
+See Also
+
+gapplyCollect
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,

[06/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/AccumulatorParam.IntAccumulatorParam$.html
--
diff --git 
a/site/docs/2.2.0/api/java/org/apache/spark/AccumulatorParam.IntAccumulatorParam$.html
 
b/site/docs/2.2.0/api/java/org/apache/spark/AccumulatorParam.IntAccumulatorParam$.html
new file mode 100644
index 000..5e91989
--- /dev/null
+++ 
b/site/docs/2.2.0/api/java/org/apache/spark/AccumulatorParam.IntAccumulatorParam$.html
@@ -0,0 +1,379 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+AccumulatorParam.IntAccumulatorParam$ (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+var methods = {"i0":42,"i1":42};
+var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"],32:["t6","Deprecated Methods"]};
+var altColor = "altColor";
+var rowColor = "rowColor";
+var tableTab = "tableTab";
+var activeTableTab = "activeTableTab";
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+
+org.apache.spark
+Class 
AccumulatorParam.IntAccumulatorParam$
+
+
+
+Object
+
+
+org.apache.spark.AccumulatorParam.IntAccumulatorParam$
+
+
+
+
+
+
+
+All Implemented Interfaces:
+java.io.Serializable, AccumulableParamObject,Object, AccumulatorParamObject
+
+
+Enclosing interface:
+AccumulatorParamT
+
+
+Deprecated.
+use AccumulatorV2. Since 
2.0.0.
+
+
+public static class AccumulatorParam.IntAccumulatorParam$
+extends Object
+implements AccumulatorParamObject
+
+See Also:
+Serialized
 Form
+
+
+
+
+
+
+
+
+
+
+
+
+Nested Class Summary
+
+
+
+
+Nested classes/interfaces inherited from 
interfaceorg.apache.spark.AccumulatorParam
+AccumulatorParam.DoubleAccumulatorParam$, 
AccumulatorParam.FloatAccumulatorParam$, 
AccumulatorParam.IntAccumulatorParam$, AccumulatorParam.LongAccumulatorParam$, 
AccumulatorParam.StringAccumulatorParam$
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields
+
+Modifier and Type
+Field and Description
+
+
+static AccumulatorParam.IntAccumulatorParam$
+MODULE$
+Deprecated.
+Static reference to the singleton instance of this Scala 
object.
+
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors
+
+Constructor and Description
+
+
+IntAccumulatorParam$()
+Deprecated.
+
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All MethodsInstance MethodsConcrete MethodsDeprecated Methods
+
+Modifier and Type
+Method and Description
+
+
+int
+addInPlace(intt1,
+  intt2)
+Deprecated.
+
+
+
+int
+zero(intinitialValue)
+Deprecated.
+
+
+
+
+
+
+
+Methods inherited from classObject
+equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, 
wait
+
+
+
+
+
+Methods inherited from interfaceorg.apache.spark.AccumulatorParam
+addAccumulator
+
+
+
+
+
+Methods inherited from interfaceorg.apache.spark.AccumulableParam
+addInPlace,
 zero
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+Field Detail
+
+
+
+
+
+MODULE$
+public static finalAccumulatorParam.IntAccumulatorParam$ 
MODULE$
+Deprecated.
+Static reference to the singleton instance of this Scala 
object.
+
+
+
+
+
+
+
+
+
+Constructor Detail
+
+
+
+
+
+IntAccumulatorParam$
+publicIntAccumulatorParam$()
+Deprecated.
+
+
+
+
+
+
+
+
+
+Method Detail
+
+
+
+
+
+addInPlace
+publicintaddInPlace(intt1,
+  intt2)
+Deprecated.
+
+
+
+
+
+
+
+zero
+publicintzero(intinitialValue)
+Deprecated.
+
+
+
+
+
+
+
+
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/AccumulatorParam.LongAccumulatorParam$.html
--
diff --git 
a/site/docs/2.2.0/api/java/org/apache/spark/AccumulatorParam.LongAccumulatorParam$.html
 
b/site/docs/2.2.0/api/java/org/apache/spark/AccumulatorParam.LongAccumulatorParam$.html
new file mode 100644
index 000..c152a36

[27/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/regexp_replace.html
--
diff --git a/site/docs/2.2.0/api/R/regexp_replace.html 
b/site/docs/2.2.0/api/R/regexp_replace.html
new file mode 100644
index 000..516a125
--- /dev/null
+++ b/site/docs/2.2.0/api/R/regexp_replace.html
@@ -0,0 +1,121 @@
+
+R: regexp_replace
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+regexp_replace {SparkR}R 
Documentation
+
+regexp_replace
+
+Description
+
+Replace all substrings of the specified string value that match regexp with 
rep.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column,character,character'
+regexp_replace(x, pattern, replacement)
+
+regexp_replace(x, pattern, replacement)
+
+
+
+Arguments
+
+
+x
+
+a string Column.
+
+pattern
+
+a regular expression.
+
+replacement
+
+a character string that a matched pattern is replaced with.
+
+
+
+
+Note
+
+regexp_replace since 1.5.0
+
+
+
+See Also
+
+Other string_funcs: ascii,
+ascii, ascii,Column-method;
+base64, base64,
+base64,Column-method;
+concat_ws, concat_ws,
+concat_ws,character,Column-method;
+concat, concat,
+concat,Column-method; decode,
+decode,
+decode,Column,character-method;
+encode, encode,
+encode,Column,character-method;
+format_number, format_number,
+format_number,Column,numeric-method;
+format_string, format_string,
+format_string,character,Column-method;
+initcap, initcap,
+initcap,Column-method; instr,
+instr,
+instr,Column,character-method;
+length, length,Column-method;
+levenshtein, levenshtein,
+levenshtein,Column-method;
+locate, locate,
+locate,character,Column-method;
+lower, lower,
+lower,Column-method; lpad,
+lpad,
+lpad,Column,numeric,character-method;
+ltrim, ltrim,
+ltrim,Column-method;
+regexp_extract,
+regexp_extract,
+regexp_extract,Column,character,numeric-method;
+reverse, reverse,
+reverse,Column-method; rpad,
+rpad,
+rpad,Column,numeric,character-method;
+rtrim, rtrim,
+rtrim,Column-method; soundex,
+soundex,
+soundex,Column-method;
+substring_index,
+substring_index,
+substring_index,Column,character,numeric-method;
+translate, translate,
+translate,Column,character,character-method;
+trim, trim,
+trim,Column-method; unbase64,
+unbase64,
+unbase64,Column-method;
+upper, upper,
+upper,Column-method
+
+
+
+Examples
+
+## Not run: regexp_replace(df$c, (\\d+), 
--)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/registerTempTable-deprecated.html
--
diff --git a/site/docs/2.2.0/api/R/registerTempTable-deprecated.html 
b/site/docs/2.2.0/api/R/registerTempTable-deprecated.html
new file mode 100644
index 000..4875838
--- /dev/null
+++ b/site/docs/2.2.0/api/R/registerTempTable-deprecated.html
@@ -0,0 +1,290 @@
+
+R: (Deprecated) Register Temporary Table
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+registerTempTable {SparkR}R 
Documentation
+
+(Deprecated) Register Temporary Table
+
+Description
+
+Registers a SparkDataFrame as a Temporary Table in the SparkSession
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,character'
+registerTempTable(x, tableName)
+
+registerTempTable(x, tableName)
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame
+
+tableName
+
+A character vector containing the name of the table
+
+
+
+
+Note
+
+registerTempTable since 1.4.0
+
+
+
+See Also
+
+createOrReplaceTempView
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,

[02/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/FetchFailed.html
--
diff --git a/site/docs/2.2.0/api/java/org/apache/spark/FetchFailed.html 
b/site/docs/2.2.0/api/java/org/apache/spark/FetchFailed.html
new file mode 100644
index 000..b12d8c1
--- /dev/null
+++ b/site/docs/2.2.0/api/java/org/apache/spark/FetchFailed.html
@@ -0,0 +1,483 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+FetchFailed (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+var methods = 
{"i0":10,"i1":5,"i2":10,"i3":5,"i4":10,"i5":10,"i6":5,"i7":5,"i8":9,"i9":9,"i10":10,"i11":10,"i12":10};
+var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],4:["t3","Abstract 
Methods"],8:["t4","Concrete Methods"]};
+var altColor = "altColor";
+var rowColor = "rowColor";
+var tableTab = "tableTab";
+var activeTableTab = "activeTableTab";
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+
+org.apache.spark
+Class FetchFailed
+
+
+
+Object
+
+
+org.apache.spark.FetchFailed
+
+
+
+
+
+
+
+All Implemented Interfaces:
+java.io.Serializable, TaskEndReason, TaskFailedReason, scala.Equals, scala.Product
+
+
+
+public class FetchFailed
+extends Object
+implements TaskFailedReason, scala.Product, 
scala.Serializable
+:: DeveloperApi ::
+ Task failed to fetch shuffle data from a remote node. Probably means we have 
lost the remote
+ executors the task is trying to fetch from, and thus need to rerun the 
previous stage.
+
+See Also:
+Serialized 
Form
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors
+
+Constructor and Description
+
+
+FetchFailed(BlockManagerIdbmAddress,
+   intshuffleId,
+   intmapId,
+   intreduceId,
+   Stringmessage)
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All MethodsStatic MethodsInstance MethodsAbstract MethodsConcrete Methods
+
+Modifier and Type
+Method and Description
+
+
+BlockManagerId
+bmAddress()
+
+
+abstract static boolean
+canEqual(Objectthat)
+
+
+boolean
+countTowardsTaskFailures()
+Fetch failures lead to a different failure handling path: 
(1) we don't abort the stage after
+ 4 task failures, instead we immediately go back to the stage which generated 
the map output,
+ and regenerate the missing data.
+
+
+
+abstract static boolean
+equals(Objectthat)
+
+
+int
+mapId()
+
+
+String
+message()
+
+
+abstract static int
+productArity()
+
+
+abstract static Object
+productElement(intn)
+
+
+static 
scala.collection.IteratorObject
+productIterator()
+
+
+static String
+productPrefix()
+
+
+int
+reduceId()
+
+
+int
+shuffleId()
+
+
+String
+toErrorString()
+Error message displayed in the web UI.
+
+
+
+
+
+
+
+Methods inherited from classObject
+equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, 
wait
+
+
+
+
+
+Methods inherited from interfacescala.Product
+productArity, productElement, productIterator, productPrefix
+
+
+
+
+
+Methods inherited from interfacescala.Equals
+canEqual, equals
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Detail
+
+
+
+
+
+FetchFailed
+publicFetchFailed(BlockManagerIdbmAddress,
+   intshuffleId,
+   intmapId,
+   intreduceId,
+   Stringmessage)
+
+
+
+
+
+
+
+
+
+Method Detail
+
+
+
+
+
+canEqual
+public abstract staticbooleancanEqual(Objectthat)
+
+
+
+
+
+
+
+equals
+public abstract staticbooleanequals(Objectthat)
+
+
+
+
+
+
+
+productElement
+public abstract staticObjectproductElement(intn)
+
+
+
+
+
+
+
+productArity
+public abstract staticintproductArity()
+
+
+
+
+
+
+
+productIterator
+public 
staticscala.collection.IteratorObjectproductIterator()
+
+
+
+
+
+
+
+productPrefix
+public staticStringproductPrefix()
+
+
+
+
+
+
+
+bmAddress
+publicBlockManagerIdbmAddress()
+
+
+
+
+
+
+
+shuffleId
+publicintshuffleId()
+
+
+
+
+
+
+
+mapId
+publicintmapId()
+
+
+
+
+
+
+
+reduceId
+publicintreduceId()
+
+
+
+
+
+
+
+message
+publicStringmessage()
+
+
+
+
+
+
+
+toErrorString
+publicStringtoErrorString()
+Description copied from 
interface:TaskFailedReason
+Error message displayed in the web UI.
+
+Specified by:
+toErrorStringin
 interfaceTaskFailedReason
+
+
+
+
+
+
+
+
+countTowardsTaskFailures
+publicbooleancountTowardsTaskFailures()
+Fetch failures 

[13/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/allclasses-frame.html
--
diff --git a/site/docs/2.2.0/api/java/allclasses-frame.html 
b/site/docs/2.2.0/api/java/allclasses-frame.html
new file mode 100644
index 000..4537763
--- /dev/null
+++ b/site/docs/2.2.0/api/java/allclasses-frame.html
@@ -0,0 +1,1196 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+All Classes (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+AllClasses
+
+
+AbsoluteError
+Accumulable
+AccumulableInfo
+AccumulableInfo
+AccumulableParam
+Accumulator
+AccumulatorContext
+AccumulatorParam
+AccumulatorParam.DoubleAccumulatorParam$
+AccumulatorParam.FloatAccumulatorParam$
+AccumulatorParam.IntAccumulatorParam$
+AccumulatorParam.LongAccumulatorParam$
+AccumulatorParam.StringAccumulatorParam$
+AccumulatorV2
+AFTAggregator
+AFTCostFun
+AFTSurvivalRegression
+AFTSurvivalRegressionModel
+AggregatedDialect
+AggregatingEdgeContext
+Aggregator
+Aggregator
+Algo
+AllJobsCancelled
+AllReceiverIds
+ALS
+ALS
+ALS.InBlock$
+ALS.Rating
+ALS.Rating$
+ALS.RatingBlock$
+ALSModel
+AnalysisException
+And
+AnyDataType
+ApplicationAttemptInfo
+ApplicationEnvironmentInfo
+ApplicationInfo
+ApplicationsListResource
+ApplicationStatus
+ApplyInPlace
+AreaUnderCurve
+ArrayType
+AskPermissionToCommitOutput
+AssociationRules
+AssociationRules
+AssociationRules.Rule
+AsyncRDDActions
+Attribute
+AttributeGroup
+AttributeKeys
+AttributeType
+BaseRelation
+BaseRRDD
+BasicBlockReplicationPolicy
+BatchInfo
+BatchInfo
+BatchStatus
+BernoulliCellSampler
+BernoulliSampler
+Binarizer
+BinaryAttribute
+BinaryClassificationEvaluator
+BinaryClassificationMetrics
+BinaryLogisticRegressionSummary
+BinaryLogisticRegressionTrainingSummary
+BinarySample
+BinaryType
+BinomialBounds
+BisectingKMeans
+BisectingKMeans
+BisectingKMeansModel
+BisectingKMeansModel
+BisectingKMeansModel.SaveLoadV1_0$
+BisectingKMeansSummary
+BlacklistedExecutor
+BLAS
+BLAS
+BlockId
+BlockManagerId
+BlockManagerMessages
+BlockManagerMessages.BlockManagerHeartbeat
+BlockManagerMessages.BlockManagerHeartbeat$
+BlockManagerMessages.GetBlockStatus
+BlockManagerMessages.GetBlockStatus$
+BlockManagerMessages.GetExecutorEndpointRef
+BlockManagerMessages.GetExecutorEndpointRef$
+BlockManagerMessages.GetLocations
+BlockManagerMessages.GetLocations$
+BlockManagerMessages.GetLocationsMultipleBlockIds
+BlockManagerMessages.GetLocationsMultipleBlockIds$
+BlockManagerMessages.GetMatchingBlockIds
+BlockManagerMessages.GetMatchingBlockIds$
+BlockManagerMessages.GetMemoryStatus$
+BlockManagerMessages.GetPeers
+BlockManagerMessages.GetPeers$
+BlockManagerMessages.GetStorageStatus$
+BlockManagerMessages.HasCachedBlocks
+BlockManagerMessages.HasCachedBlocks$
+BlockManagerMessages.RegisterBlockManager
+BlockManagerMessages.RegisterBlockManager$
+BlockManagerMessages.RemoveBlock
+BlockManagerMessages.RemoveBlock$
+BlockManagerMessages.RemoveBroadcast
+BlockManagerMessages.RemoveBroadcast$
+BlockManagerMessages.RemoveExecutor
+BlockManagerMessages.RemoveExecutor$
+BlockManagerMessages.RemoveRdd
+BlockManagerMessages.RemoveRdd$
+BlockManagerMessages.RemoveShuffle
+BlockManagerMessages.RemoveShuffle$
+BlockManagerMessages.ReplicateBlock
+BlockManagerMessages.ReplicateBlock$
+BlockManagerMessages.StopBlockManagerMaster$
+BlockManagerMessages.ToBlockManagerMaster
+BlockManagerMessages.ToBlockManagerSlave
+BlockManagerMessages.TriggerThreadDump$
+BlockManagerMessages.UpdateBlockInfo
+BlockManagerMessages.UpdateBlockInfo$
+BlockMatrix
+BlockNotFoundException
+BlockReplicationPolicy
+BlockReplicationUtils
+BlockStatus
+BlockUpdatedInfo
+BloomFilter
+BloomFilter.Version
+BooleanParam
+BooleanType
+BoostingStrategy
+BoundedDouble
+BreezeUtil
+Broadcast
+BroadcastBlockId
+Broker
+BucketedRandomProjectionLSH
+BucketedRandomProjectionLSHModel
+Bucketizer
+BufferReleasingInputStream
+BytecodeUtils
+ByteType
+CalendarIntervalType
+Catalog
+CatalystScan
+CategoricalSplit
+CausedBy
+CharType
+CheckpointReader
+CheckpointState
+ChiSqSelector
+ChiSqSelector
+ChiSqSelectorModel
+ChiSqSelectorModel
+ChiSqSelectorModel.SaveLoadV1_0$
+ChiSqTest
+ChiSqTest.Method
+ChiSqTest.Method$
+ChiSqTest.NullHypothesis$
+ChiSqTestResult
+ChiSquareTest
+CholeskyDecomposition
+ClassificationModel
+ClassificationModel
+Classifier
+CleanAccum
+CleanBroadcast
+CleanCheckpoint
+CleanRDD
+CleanShuffle
+CleanupTask
+CleanupTaskWeakReference
+ClosureCleaner
+ClusteringSummary
+CoarseGrainedClusterMessages
+CoarseGrainedClusterMessages.AddWebUIFilter
+CoarseGrainedClusterMessages.AddWebUIFilter$
+CoarseGrainedClusterMessages.GetExecutorLossReason
+CoarseGrainedClusterMessages.GetExecutorLossReason$
+CoarseGrainedClusterMessages.KillExecutors
+CoarseGrainedClusterMessages.KillExecutors$
+CoarseGrainedClusterMessages.KillExecutorsOnHost
+CoarseGrainedClusterMessages.KillExecutorsOnHost$
+CoarseGrainedClusterMessages.KillTask
+CoarseGrainedClusterMessages.KillTask$

[18/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/to_json.html
--
diff --git a/site/docs/2.2.0/api/R/to_json.html 
b/site/docs/2.2.0/api/R/to_json.html
new file mode 100644
index 000..77fa20d
--- /dev/null
+++ b/site/docs/2.2.0/api/R/to_json.html
@@ -0,0 +1,112 @@
+
+R: to_json
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+to_json 
{SparkR}R Documentation
+
+to_json
+
+Description
+
+Converts a column containing a structType or array of 
structType into a Column
+of JSON string. Resolving the Column can fail if an unsupported type is 
encountered.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+to_json(x, ...)
+
+to_json(x, ...)
+
+
+
+Arguments
+
+
+x
+
+Column containing the struct or array of the structs
+
+...
+
+additional named properties to control how it is converted, accepts the 
same options
+as the JSON data source.
+
+
+
+
+Note
+
+to_json since 2.2.0
+
+
+
+See Also
+
+Other normal_funcs: Column-class,
+column, column,
+column,
+column,character-method,
+column,jobj-method; abs,
+abs,Column-method;
+bitwiseNOT, bitwiseNOT,
+bitwiseNOT,Column-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+expr, expr,
+expr,character-method;
+from_json, from_json,
+from_json,Column,structType-method;
+greatest, greatest,
+greatest,Column-method;
+ifelse, ifelse,Column-method;
+is.nan, is.nan,Column-method,
+isnan, isnan,
+isnan,Column-method; least,
+least, least,Column-method;
+lit, lit,
+lit,ANY-method; nanvl,
+nanvl, nanvl,Column-method;
+negate, negate,
+negate,Column-method; randn,
+randn, randn,
+randn,missing-method,
+randn,numeric-method; rand,
+rand, rand,
+rand,missing-method,
+rand,numeric-method; struct,
+struct,
+struct,characterOrColumn-method;
+when, when,
+when,Column-method
+
+
+
+Examples
+
+## Not run: 
+##D # Converts a struct into a JSON object
+##D df - sql(SELECT named_struct(date, 
cast(2000-01-01 as date)) as d)
+##D select(df, to_json(df$d, dateFormat = dd/MM/))
+##D 
+##D # Converts an array of structs into a JSON array
+##D df - sql(SELECT array(named_struct(name, 
Bob), named_struct(name, Alice)) as people)
+##D select(df, to_json(df$people))
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/to_timestamp.html
--
diff --git a/site/docs/2.2.0/api/R/to_timestamp.html 
b/site/docs/2.2.0/api/R/to_timestamp.html
new file mode 100644
index 000..6b76614
--- /dev/null
+++ b/site/docs/2.2.0/api/R/to_timestamp.html
@@ -0,0 +1,190 @@
+
+R: to_timestamp
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+to_timestamp {SparkR}R 
Documentation
+
+to_timestamp
+
+Description
+
+Converts the column into a TimestampType. You may optionally specify a 
format
+according to the rules in:
+http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html;>http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html.
+If the string cannot be parsed according to the specified format (or default),
+the value of the column will be null.
+The default format is '-MM-dd HH:mm:ss'.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column,missing'
+to_timestamp(x, format)
+
+## S4 method for signature 'Column,character'
+to_timestamp(x, format)
+
+to_timestamp(x, format)
+
+
+
+Arguments
+
+
+x
+
+Column to parse.
+
+format
+
+string to use to parse x Column to DateType. (optional)
+
+
+
+
+Note
+
+to_timestamp(Column) since 2.2.0
+
+to_timestamp(Column, character) since 2.2.0
+
+
+
+See Also
+
+Other datetime_funcs: add_months,
+add_months,
+add_months,Column,numeric-method;
+date_add, date_add,
+date_add,Column,numeric-method;
+date_format, date_format,
+date_format,Column,character-method;
+date_sub, date_sub,
+date_sub,Column,numeric-method;
+datediff, datediff,
+datediff,Column-method;
+dayofmonth, dayofmonth,
+dayofmonth,Column-method;
+dayofyear, dayofyear,
+dayofyear,Column-method;
+from_unixtime, from_unixtime,
+from_unixtime,Column-method;
+from_utc_timestamp,
+from_utc_timestamp,
+from_utc_timestamp,Column,character-method;
+hour, hour,
+hour,Column-method; last_day,
+last_day,
+last_day,Column-method;
+minute, minute,
+minute,Column-method;
+months_between,
+months_between,
+months_between,Column-method;
+month, month,
+month,Column-method;
+next_day, 

[44/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/coltypes.html
--
diff --git a/site/docs/2.2.0/api/R/coltypes.html 
b/site/docs/2.2.0/api/R/coltypes.html
new file mode 100644
index 000..39a73ca
--- /dev/null
+++ b/site/docs/2.2.0/api/R/coltypes.html
@@ -0,0 +1,308 @@
+
+R: coltypes
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+coltypes 
{SparkR}R Documentation
+
+coltypes
+
+Description
+
+Get column types of a SparkDataFrame
+
+Set the column types of a SparkDataFrame.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+coltypes(x)
+
+## S4 replacement method for signature 'SparkDataFrame,character'
+coltypes(x) - value
+
+coltypes(x)
+
+coltypes(x) - value
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame
+
+value
+
+A character vector with the target column types for the given
+SparkDataFrame. Column types can be one of integer, numeric/double, character, 
logical, or NA
+to keep that column as-is.
+
+
+
+
+Value
+
+value A character vector with the column types of the given SparkDataFrame
+
+
+
+Note
+
+coltypes since 1.6.0
+
+coltypes- since 1.6.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+intersect, intersect,
+intersect,SparkDataFrame,SparkDataFrame-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,

[01/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
Repository: spark-website
Updated Branches:
  refs/heads/asf-site 04d5ce051 -> ee654d1f3


http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/InternalAccumulator.output$.html
--
diff --git 
a/site/docs/2.2.0/api/java/org/apache/spark/InternalAccumulator.output$.html 
b/site/docs/2.2.0/api/java/org/apache/spark/InternalAccumulator.output$.html
new file mode 100644
index 000..a04523b
--- /dev/null
+++ b/site/docs/2.2.0/api/java/org/apache/spark/InternalAccumulator.output$.html
@@ -0,0 +1,325 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+InternalAccumulator.output$ (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+var methods = {"i0":10,"i1":10};
+var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
+var altColor = "altColor";
+var rowColor = "rowColor";
+var tableTab = "tableTab";
+var activeTableTab = "activeTableTab";
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+
+org.apache.spark
+Class 
InternalAccumulator.output$
+
+
+
+Object
+
+
+org.apache.spark.InternalAccumulator.output$
+
+
+
+
+
+
+
+Enclosing class:
+InternalAccumulator
+
+
+
+public static class InternalAccumulator.output$
+extends Object
+
+
+
+
+
+
+
+
+
+
+
+Field Summary
+
+Fields
+
+Modifier and Type
+Field and Description
+
+
+static InternalAccumulator.output$
+MODULE$
+Static reference to the singleton instance of this Scala 
object.
+
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors
+
+Constructor and Description
+
+
+output$()
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All MethodsInstance MethodsConcrete Methods
+
+Modifier and Type
+Method and Description
+
+
+String
+BYTES_WRITTEN()
+
+
+String
+RECORDS_WRITTEN()
+
+
+
+
+
+
+Methods inherited from classObject
+equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, 
wait
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+Field Detail
+
+
+
+
+
+MODULE$
+public static finalInternalAccumulator.output$ MODULE$
+Static reference to the singleton instance of this Scala 
object.
+
+
+
+
+
+
+
+
+
+Constructor Detail
+
+
+
+
+
+output$
+publicoutput$()
+
+
+
+
+
+
+
+
+
+Method Detail
+
+
+
+
+
+BYTES_WRITTEN
+publicStringBYTES_WRITTEN()
+
+
+
+
+
+
+
+RECORDS_WRITTEN
+publicStringRECORDS_WRITTEN()
+
+
+
+
+
+
+
+
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+


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



[53/56] spark-website git commit: Rebuild for 2.2.0

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/screencasts/3-transformations-and-caching.html
--
diff --git a/site/screencasts/3-transformations-and-caching.html 
b/site/screencasts/3-transformations-and-caching.html
index 38cd5cc..fdcc294 100644
--- a/site/screencasts/3-transformations-and-caching.html
+++ b/site/screencasts/3-transformations-and-caching.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/screencasts/4-a-standalone-job-in-spark.html
--
diff --git a/site/screencasts/4-a-standalone-job-in-spark.html 
b/site/screencasts/4-a-standalone-job-in-spark.html
index 7ff3ff1..535471e 100644
--- a/site/screencasts/4-a-standalone-job-in-spark.html
+++ b/site/screencasts/4-a-standalone-job-in-spark.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/screencasts/index.html
--
diff --git a/site/screencasts/index.html b/site/screencasts/index.html
index f6310f5..a2d4c28 100644
--- a/site/screencasts/index.html
+++ b/site/screencasts/index.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/security.html
--
diff --git a/site/security.html b/site/security.html
index 3870dbe..83345d3 100644
--- a/site/security.html
+++ b/site/security.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/sitemap.xml
--
diff --git a/site/sitemap.xml b/site/sitemap.xml
index 6748fd0..591e871 100644
--- a/site/sitemap.xml
+++ b/site/sitemap.xml
@@ -139,6 +139,14 @@
 
 
 
+  https://spark.apache.org/releases/spark-release-2-2-0.html
+  weekly
+
+
+  https://spark.apache.org/news/spark-2-2-0-released.html
+  weekly
+
+
   https://spark.apache.org/releases/spark-release-2-1-1.html
   weekly
 
@@ -644,11 +652,11 @@
   weekly
 
 
-  https://spark.apache.org/graphx/
+  https://spark.apache.org/sql/
   weekly
 
 
-  https://spark.apache.org/sql/
+  https://spark.apache.org/streaming/
   weekly
 
 
@@ -656,19 +664,19 @@
   weekly
 
 
-  https://spark.apache.org/streaming/
+  

[26/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/round.html
--
diff --git a/site/docs/2.2.0/api/R/round.html b/site/docs/2.2.0/api/R/round.html
new file mode 100644
index 000..35b3cbf
--- /dev/null
+++ b/site/docs/2.2.0/api/R/round.html
@@ -0,0 +1,120 @@
+
+R: round
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+round 
{SparkR}R Documentation
+
+round
+
+Description
+
+Returns the value of the column e rounded to 0 decimal places 
using HALF_UP rounding mode.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+round(x)
+
+
+
+Arguments
+
+
+x
+
+Column to compute on.
+
+
+
+
+Note
+
+round since 1.5.0
+
+
+
+See Also
+
+Other math_funcs: acos,
+acos,Column-method; asin,
+asin,Column-method; atan2,
+atan2,Column-method; atan,
+atan,Column-method; bin,
+bin, bin,Column-method;
+bround, bround,
+bround,Column-method; cbrt,
+cbrt, cbrt,Column-method;
+ceil, ceil,
+ceil,Column-method, ceiling,
+ceiling,Column-method; conv,
+conv,
+conv,Column,numeric,numeric-method;
+corr, corr,
+corr, corr,Column-method,
+corr,SparkDataFrame-method;
+cosh, cosh,Column-method;
+cos, cos,Column-method;
+covar_pop, covar_pop,
+covar_pop,characterOrColumn,characterOrColumn-method;
+cov, cov, cov,
+cov,SparkDataFrame-method,
+cov,characterOrColumn-method,
+covar_samp, covar_samp,
+covar_samp,characterOrColumn,characterOrColumn-method;
+expm1, expm1,Column-method;
+exp, exp,Column-method;
+factorial,
+factorial,Column-method;
+floor, floor,Column-method;
+hex, hex,
+hex,Column-method; hypot,
+hypot, hypot,Column-method;
+log10, log10,Column-method;
+log1p, log1p,Column-method;
+log2, log2,Column-method;
+log, log,Column-method;
+pmod, pmod,
+pmod,Column-method; rint,
+rint, rint,Column-method;
+shiftLeft, shiftLeft,
+shiftLeft,Column,numeric-method;
+shiftRightUnsigned,
+shiftRightUnsigned,
+shiftRightUnsigned,Column,numeric-method;
+shiftRight, shiftRight,
+shiftRight,Column,numeric-method;
+sign, sign,Column-method,
+signum, signum,
+signum,Column-method; sinh,
+sinh,Column-method; sin,
+sin,Column-method; sqrt,
+sqrt,Column-method; tanh,
+tanh,Column-method; tan,
+tan,Column-method; toDegrees,
+toDegrees,
+toDegrees,Column-method;
+toRadians, toRadians,
+toRadians,Column-method;
+unhex, unhex,
+unhex,Column-method
+
+
+
+Examples
+
+## Not run: round(df$c)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/row_number.html
--
diff --git a/site/docs/2.2.0/api/R/row_number.html 
b/site/docs/2.2.0/api/R/row_number.html
new file mode 100644
index 000..07dff91
--- /dev/null
+++ b/site/docs/2.2.0/api/R/row_number.html
@@ -0,0 +1,86 @@
+
+R: row_number
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+row_number 
{SparkR}R Documentation
+
+row_number
+
+Description
+
+Window function: returns a sequential number starting at 1 within a window 
partition.
+
+
+
+Usage
+
+
+## S4 method for signature 'missing'
+row_number()
+
+row_number(x = "missing")
+
+
+
+Arguments
+
+
+x
+
+empty. Should be used with no argument.
+
+
+
+
+Details
+
+This is equivalent to the ROW_NUMBER function in SQL.
+
+
+
+Note
+
+row_number since 1.6.0
+
+
+
+See Also
+
+Other window_funcs: cume_dist,
+cume_dist,
+cume_dist,missing-method;
+dense_rank, dense_rank,
+dense_rank,missing-method;
+lag, lag,
+lag,characterOrColumn-method;
+lead, lead,
+lead,characterOrColumn,numeric-method;
+ntile, ntile,
+ntile,numeric-method;
+percent_rank, percent_rank,
+percent_rank,missing-method;
+rank, rank,
+rank, rank,ANY-method,
+rank,missing-method
+
+
+
+Examples
+
+## Not run: 
+##D   df - createDataFrame(mtcars)
+##D   ws - orderBy(windowPartitionBy(am), hp)
+##D   out - select(df, over(row_number(), ws), df$hp, df$am)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/rowsBetween.html
--
diff --git a/site/docs/2.2.0/api/R/rowsBetween.html 
b/site/docs/2.2.0/api/R/rowsBetween.html
new file mode 100644
index 000..73d8d23
--- /dev/null
+++ b/site/docs/2.2.0/api/R/rowsBetween.html
@@ -0,0 +1,94 @@
+
+R: rowsBetween
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>

[09/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/index.html
--
diff --git a/site/docs/2.2.0/api/java/index.html 
b/site/docs/2.2.0/api/java/index.html
new file mode 100644
index 000..d40ae69
--- /dev/null
+++ b/site/docs/2.2.0/api/java/index.html
@@ -0,0 +1,74 @@
+http://www.w3.org/TR/html4/frameset.dtd;>
+
+
+
+
+Spark 2.2.0 JavaDoc
+
+targetPage = "" + window.location.search;
+if (targetPage != "" && targetPage != "undefined")
+targetPage = targetPage.substring(1);
+if (targetPage.indexOf(":") != -1 || (targetPage != "" && 
!validURL(targetPage)))
+targetPage = "undefined";
+function validURL(url) {
+try {
+url = decodeURIComponent(url);
+}
+catch (error) {
+return false;
+}
+var pos = url.indexOf(".html");
+if (pos == -1 || pos != url.length - 5)
+return false;
+var allowNumber = false;
+var allowSep = false;
+var seenDot = false;
+for (var i = 0; i < url.length - 5; i++) {
+var ch = url.charAt(i);
+if ('a' <= ch && ch <= 'z' ||
+'A' <= ch && ch <= 'Z' ||
+ch == '$' ||
+ch == '_' ||
+ch.charCodeAt(0) > 127) {
+allowNumber = true;
+allowSep = true;
+} else if ('0' <= ch && ch <= '9'
+|| ch == '-') {
+if (!allowNumber)
+ return false;
+} else if (ch == '/' || ch == '.') {
+if (!allowSep)
+return false;
+allowNumber = false;
+allowSep = false;
+if (ch == '.')
+ seenDot = true;
+if (ch == '/' && seenDot)
+ return false;
+} else {
+return false;
+}
+}
+return true;
+}
+function loadFrames() {
+if (targetPage != "" && targetPage != "undefined")
+ top.classFrame.location = top.targetPage;
+}
+
+
+
+
+
+
+
+
+
+
+JavaScript is disabled on your browser.
+
+Frame Alert
+This document is designed to be viewed using the frames feature. If you see 
this message, you are using a non-frame-capable web client. Link to Non-frame version.
+
+
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/lib/api-javadocs.js
--
diff --git a/site/docs/2.2.0/api/java/lib/api-javadocs.js 
b/site/docs/2.2.0/api/java/lib/api-javadocs.js
new file mode 100644
index 000..ead13d6
--- /dev/null
+++ b/site/docs/2.2.0/api/java/lib/api-javadocs.js
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+
+/* Dynamically injected post-processing code for the API docs */
+
+$(document).ready(function() {
+  addBadges(":: AlphaComponent ::", 'Alpha 
Component');
+  addBadges(":: DeveloperApi ::", 'Developer 
API');
+  addBadges(":: Experimental ::", 'Experimental');
+});
+
+function addBadges(tag, html) {
+  var tags = $(".block:contains(" + tag + ")")
+
+  // Remove identifier tags
+  tags.each(function(index) {
+var oldHTML = $(this).html();
+var newHTML = oldHTML.replace(tag, "");
+$(this).html(newHTML);
+  });
+
+  // Add html badge tags
+  tags.each(function(index) {
+if ($(this).parent().is('td.colLast')) {
+  $(this).parent().prepend(html);
+} else if ($(this).parent('li.blockList')
+  .parent('ul.blockList')
+  .parent('div.description')
+  .parent().is('div.contentContainer')) {
+  var contentContainer = $(this).parent('li.blockList')
+.parent('ul.blockList')
+.parent('div.description')
+.parent('div.contentContainer')
+  var header = contentContainer.prev('div.header');
+  if (header.length > 0) {
+header.prepend(html);
+  } else {
+

[50/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/README.md
--
diff --git a/site/docs/2.2.0/README.md b/site/docs/2.2.0/README.md
new file mode 100644
index 000..90e10a1
--- /dev/null
+++ b/site/docs/2.2.0/README.md
@@ -0,0 +1,73 @@
+Welcome to the Spark documentation!
+
+This readme will walk you through navigating and building the Spark 
documentation, which is included
+here with the Spark source code. You can also find documentation specific to 
release versions of
+Spark at http://spark.apache.org/documentation.html.
+
+Read on to learn more about viewing documentation in plain text (i.e., 
markdown) or building the
+documentation yourself. Why build it yourself? So that you have the docs that 
corresponds to
+whichever version of Spark you currently have checked out of revision control.
+
+## Prerequisites
+The Spark documentation build uses a number of tools to build HTML docs and 
API docs in Scala,
+Python and R.
+
+You need to have 
[Ruby](https://www.ruby-lang.org/en/documentation/installation/) and
+[Python](https://docs.python.org/2/using/unix.html#getting-and-installing-the-latest-version-of-python)
+installed. Also install the following libraries:
+```sh
+$ sudo gem install jekyll jekyll-redirect-from pygments.rb
+$ sudo pip install Pygments
+# Following is needed only for generating API docs
+$ sudo pip install sphinx pypandoc
+$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "roxygen2", 
"testthat", "rmarkdown"), repos="http://cran.stat.ucla.edu/;)'
+```
+(Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to 
replace gem with gem2.0)
+
+## Generating the Documentation HTML
+
+We include the Spark documentation as part of the source (as opposed to using 
a hosted wiki, such as
+the github wiki, as the definitive documentation) to enable the documentation 
to evolve along with
+the source code and be captured by revision control (currently git). This way 
the code automatically
+includes the version of the documentation that is relevant regardless of which 
version or release
+you have checked out or downloaded.
+
+In this directory you will find textfiles formatted using Markdown, with an 
".md" suffix. You can
+read those text files directly if you want. Start with index.md.
+
+Execute `jekyll build` from the `docs/` directory to compile the site. 
Compiling the site with
+Jekyll will create a directory called `_site` containing index.html as well as 
the rest of the
+compiled files.
+
+$ cd docs
+$ jekyll build
+
+You can modify the default Jekyll build as follows:
+```sh
+# Skip generating API docs (which takes a while)
+$ SKIP_API=1 jekyll build
+
+# Serve content locally on port 4000
+$ jekyll serve --watch
+
+# Build the site with extra features used on the live page
+$ PRODUCTION=1 jekyll build
+```
+
+## API Docs (Scaladoc, Sphinx, roxygen2)
+
+You can build just the Spark scaladoc by running `build/sbt unidoc` from the 
SPARK_PROJECT_ROOT directory.
+
+Similarly, you can build just the PySpark docs by running `make html` from the
+SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for 
classes that are listed as
+public in `__init__.py`. The SparkR docs can be built by running 
SPARK_PROJECT_ROOT/R/create-docs.sh.
+
+When you run `jekyll` in the `docs` directory, it will also copy over the 
scaladoc for the various
+Spark subprojects into the `docs` directory (and then also into the `_site` 
directory). We use a
+jekyll plugin to run `build/sbt unidoc` before building the site so if you 
haven't run it (recently) it
+may take some time as it generates all of the scaladoc.  The jekyll plugin 
also generates the
+PySpark docs using [Sphinx](http://sphinx-doc.org/).
+
+NOTE: To skip the step of building and copying over the Scala, Python, R API 
docs, run `SKIP_API=1
+jekyll`. In addition, `SKIP_SCALADOC=1`, `SKIP_PYTHONDOC=1`, and `SKIP_RDOC=1` 
can be used to skip a single
+step of the corresponding language.

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api.html
--
diff --git a/site/docs/2.2.0/api.html b/site/docs/2.2.0/api.html
new file mode 100644
index 000..835f32d
--- /dev/null
+++ b/site/docs/2.2.0/api.html
@@ -0,0 +1,178 @@
+
+
+
+
+
+  
+
+
+
+Spark API Documentation - Spark 2.2.0 Documentation
+
+
+
+
+
+
+body {
+padding-top: 60px;
+padding-bottom: 40px;
+}
+
+
+
+
+
+
+
+
+
+
+
+
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-32518208-2']);
+  _gaq.push(['_trackPageview']);
+
+  (function() {
+var ga = 

[05/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/Aggregator.html
--
diff --git a/site/docs/2.2.0/api/java/org/apache/spark/Aggregator.html 
b/site/docs/2.2.0/api/java/org/apache/spark/Aggregator.html
new file mode 100644
index 000..c916b4a
--- /dev/null
+++ b/site/docs/2.2.0/api/java/org/apache/spark/Aggregator.html
@@ -0,0 +1,436 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+Aggregator (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+var methods = 
{"i0":5,"i1":10,"i2":10,"i3":10,"i4":5,"i5":10,"i6":10,"i7":5,"i8":5,"i9":9,"i10":9};
+var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],4:["t3","Abstract 
Methods"],8:["t4","Concrete Methods"]};
+var altColor = "altColor";
+var rowColor = "rowColor";
+var tableTab = "tableTab";
+var activeTableTab = "activeTableTab";
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+
+org.apache.spark
+Class AggregatorK,V,C
+
+
+
+Object
+
+
+org.apache.spark.AggregatorK,V,C
+
+
+
+
+
+
+
+All Implemented Interfaces:
+java.io.Serializable, scala.Equals, scala.Product
+
+
+
+public class AggregatorK,V,C
+extends Object
+implements scala.Product, scala.Serializable
+:: DeveloperApi ::
+ A set of functions used to aggregate data.
+ 
+ param:  createCombiner function to create the initial value of the 
aggregation.
+ param:  mergeValue function to merge a new value into the aggregation result.
+ param:  mergeCombiners function to merge outputs from multiple mergeValue 
function.
+
+See Also:
+Serialized 
Form
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors
+
+Constructor and Description
+
+
+Aggregator(scala.Function1V,CcreateCombiner,
+  scala.Function2C,V,CmergeValue,
+  scala.Function2C,C,CmergeCombiners)
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All MethodsStatic MethodsInstance MethodsAbstract MethodsConcrete Methods
+
+Modifier and Type
+Method and Description
+
+
+abstract static boolean
+canEqual(Objectthat)
+
+
+scala.collection.Iteratorscala.Tuple2K,C
+combineCombinersByKey(scala.collection.Iterator?
 extends scala.Product2K,Citer,
+ TaskContextcontext)
+
+
+scala.collection.Iteratorscala.Tuple2K,C
+combineValuesByKey(scala.collection.Iterator?
 extends scala.Product2K,Viter,
+  TaskContextcontext)
+
+
+scala.Function1V,C
+createCombiner()
+
+
+abstract static boolean
+equals(Objectthat)
+
+
+scala.Function2C,C,C
+mergeCombiners()
+
+
+scala.Function2C,V,C
+mergeValue()
+
+
+abstract static int
+productArity()
+
+
+abstract static Object
+productElement(intn)
+
+
+static 
scala.collection.IteratorObject
+productIterator()
+
+
+static String
+productPrefix()
+
+
+
+
+
+
+Methods inherited from classObject
+equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, 
wait
+
+
+
+
+
+Methods inherited from interfacescala.Product
+productArity, productElement, productIterator, productPrefix
+
+
+
+
+
+Methods inherited from interfacescala.Equals
+canEqual, equals
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Detail
+
+
+
+
+
+Aggregator
+publicAggregator(scala.Function1V,CcreateCombiner,
+  scala.Function2C,V,CmergeValue,
+  scala.Function2C,C,CmergeCombiners)
+
+
+
+
+
+
+
+
+
+Method Detail
+
+
+
+
+
+canEqual
+public abstract staticbooleancanEqual(Objectthat)
+
+
+
+
+
+
+
+equals
+public abstract staticbooleanequals(Objectthat)
+
+
+
+
+
+
+
+productElement
+public abstract staticObjectproductElement(intn)
+
+
+
+
+
+
+
+productArity
+public abstract staticintproductArity()
+
+
+
+
+
+
+
+productIterator
+public 
staticscala.collection.IteratorObjectproductIterator()
+
+
+
+
+
+
+
+productPrefix
+public staticStringproductPrefix()
+
+
+
+
+
+
+
+createCombiner
+publicscala.Function1V,CcreateCombiner()
+
+
+
+
+
+
+
+mergeValue
+publicscala.Function2C,V,CmergeValue()
+
+
+
+
+
+
+
+mergeCombiners
+publicscala.Function2C,C,CmergeCombiners()
+
+
+
+
+
+
+
+combineValuesByKey
+publicscala.collection.Iteratorscala.Tuple2K,CcombineValuesByKey(scala.collection.Iterator?
 extends scala.Product2K,Viter,
+   TaskContextcontext)
+
+
+
+
+
+
+
+combineCombinersByKey

[28/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/randn.html
--
diff --git a/site/docs/2.2.0/api/R/randn.html b/site/docs/2.2.0/api/R/randn.html
new file mode 100644
index 000..ed45118
--- /dev/null
+++ b/site/docs/2.2.0/api/R/randn.html
@@ -0,0 +1,102 @@
+
+R: randn
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+randn 
{SparkR}R Documentation
+
+randn
+
+Description
+
+Generate a column with independent and identically distributed (i.i.d.) 
samples from
+the standard normal distribution.
+
+
+
+Usage
+
+
+## S4 method for signature 'missing'
+randn(seed)
+
+## S4 method for signature 'numeric'
+randn(seed)
+
+randn(seed)
+
+
+
+Arguments
+
+
+seed
+
+a random seed. Can be missing.
+
+
+
+
+Note
+
+randn since 1.5.0
+
+randn(numeric) since 1.5.0
+
+
+
+See Also
+
+Other normal_funcs: Column-class,
+column, column,
+column,
+column,character-method,
+column,jobj-method; abs,
+abs,Column-method;
+bitwiseNOT, bitwiseNOT,
+bitwiseNOT,Column-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+expr, expr,
+expr,character-method;
+from_json, from_json,
+from_json,Column,structType-method;
+greatest, greatest,
+greatest,Column-method;
+ifelse, ifelse,Column-method;
+is.nan, is.nan,Column-method,
+isnan, isnan,
+isnan,Column-method; least,
+least, least,Column-method;
+lit, lit,
+lit,ANY-method; nanvl,
+nanvl, nanvl,Column-method;
+negate, negate,
+negate,Column-method; rand,
+rand, rand,
+rand,missing-method,
+rand,numeric-method; struct,
+struct,
+struct,characterOrColumn-method;
+to_json, to_json,
+to_json,Column-method; when,
+when, when,Column-method
+
+
+
+Examples
+
+## Not run: randn()
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/randomSplit.html
--
diff --git a/site/docs/2.2.0/api/R/randomSplit.html 
b/site/docs/2.2.0/api/R/randomSplit.html
new file mode 100644
index 000..8bc3122
--- /dev/null
+++ b/site/docs/2.2.0/api/R/randomSplit.html
@@ -0,0 +1,293 @@
+
+R: randomSplit
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+randomSplit {SparkR}R 
Documentation
+
+randomSplit
+
+Description
+
+Return a list of randomly split dataframes with the provided weights.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,numeric'
+randomSplit(x, weights, seed)
+
+randomSplit(x, weights, seed)
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame
+
+weights
+
+A vector of weights for splits, will be normalized if they don't sum to 
1
+
+seed
+
+A seed to use for random split
+
+
+
+
+Note
+
+randomSplit since 2.0.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, 

[39/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/explain.html
--
diff --git a/site/docs/2.2.0/api/R/explain.html 
b/site/docs/2.2.0/api/R/explain.html
new file mode 100644
index 000..857150f
--- /dev/null
+++ b/site/docs/2.2.0/api/R/explain.html
@@ -0,0 +1,310 @@
+
+R: Explain
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+explain 
{SparkR}R Documentation
+
+Explain
+
+Description
+
+Print the logical and physical Catalyst plans to the console for debugging.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+explain(x, extended = FALSE)
+
+explain(x, ...)
+
+## S4 method for signature 'StreamingQuery'
+explain(x, extended = FALSE)
+
+
+
+Arguments
+
+
+x
+
+a SparkDataFrame or a StreamingQuery.
+
+extended
+
+Logical. If extended is FALSE, prints only the physical plan.
+
+...
+
+further arguments to be passed to or from other methods.
+
+
+
+
+Note
+
+explain since 1.4.0
+
+explain(StreamingQuery) since 2.2.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+intersect, intersect,
+intersect,SparkDataFrame,SparkDataFrame-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,
+join,SparkDataFrame,SparkDataFrame-method;
+limit, limit,
+limit,SparkDataFrame,numeric-method;
+merge, merge,

[40/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/dropDuplicates.html
--
diff --git a/site/docs/2.2.0/api/R/dropDuplicates.html 
b/site/docs/2.2.0/api/R/dropDuplicates.html
new file mode 100644
index 000..2b753a2
--- /dev/null
+++ b/site/docs/2.2.0/api/R/dropDuplicates.html
@@ -0,0 +1,297 @@
+
+R: dropDuplicates
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+dropDuplicates {SparkR}R 
Documentation
+
+dropDuplicates
+
+Description
+
+Returns a new SparkDataFrame with duplicate rows removed, considering only
+the subset of columns.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+dropDuplicates(x, ...)
+
+dropDuplicates(x, ...)
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame.
+
+...
+
+A character vector of column names or string column names.
+If the first argument contains a character vector, the followings are 
ignored.
+
+
+
+
+Value
+
+A SparkDataFrame with duplicate rows removed.
+
+
+
+Note
+
+dropDuplicates since 2.0.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+intersect, intersect,
+intersect,SparkDataFrame,SparkDataFrame-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,
+join,SparkDataFrame,SparkDataFrame-method;
+limit, limit,

[20/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/subset.html
--
diff --git a/site/docs/2.2.0/api/R/subset.html 
b/site/docs/2.2.0/api/R/subset.html
new file mode 100644
index 000..0fef86d
--- /dev/null
+++ b/site/docs/2.2.0/api/R/subset.html
@@ -0,0 +1,351 @@
+
+R: Subset
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+[[ {SparkR}R Documentation
+
+Subset
+
+Description
+
+Return subsets of SparkDataFrame according to given conditions
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,numericOrcharacter'
+x[[i]]
+
+## S4 replacement method for signature 'SparkDataFrame,numericOrcharacter'
+x[[i]] - value
+
+## S4 method for signature 'SparkDataFrame'
+x[i, j, ..., drop = F]
+
+## S4 method for signature 'SparkDataFrame'
+subset(x, subset, select, drop = F, ...)
+
+subset(x, ...)
+
+
+
+Arguments
+
+
+x
+
+a SparkDataFrame.
+
+i,subset
+
+(Optional) a logical expression to filter on rows.
+For extract operator [[ and replacement operator [[-, the indexing 
parameter for
+a single Column.
+
+value
+
+a Column or an atomic vector in the length of 1 as literal value, or 
NULL.
+If NULL, the specified Column is dropped.
+
+j,select
+
+expression for the single Column or a list of columns to select from the 
SparkDataFrame.
+
+...
+
+currently not used.
+
+drop
+
+if TRUE, a Column will be returned if the resulting dataset has only one 
column.
+Otherwise, a SparkDataFrame will always be returned.
+
+
+
+
+Value
+
+A new SparkDataFrame containing only the rows that meet the condition with 
selected columns.
+
+
+
+Note
+
+[[ since 1.4.0
+
+[[- since 2.1.1
+
+[ since 1.4.0
+
+subset since 1.5.0
+
+
+
+See Also
+
+withColumn
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; agg,
+agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,

[31/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/merge.html
--
diff --git a/site/docs/2.2.0/api/R/merge.html b/site/docs/2.2.0/api/R/merge.html
new file mode 100644
index 000..0bc3629
--- /dev/null
+++ b/site/docs/2.2.0/api/R/merge.html
@@ -0,0 +1,354 @@
+
+R: Merges two data frames
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+merge 
{SparkR}R Documentation
+
+Merges two data frames
+
+Description
+
+Merges two data frames
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,SparkDataFrame'
+merge(x, y, by = intersect(names(x),
+  names(y)), by.x = by, by.y = by, all = FALSE, all.x = all,
+  all.y = all, sort = TRUE, suffixes = c("_x", "_y"), ...)
+
+merge(x, y, ...)
+
+
+
+Arguments
+
+
+x
+
+the first data frame to be joined.
+
+y
+
+the second data frame to be joined.
+
+by
+
+a character vector specifying the join columns. If by is not
+specified, the common column names in x and y will 
be used.
+If by or both by.x and by.y are explicitly set to NULL or of length 0, the 
Cartesian
+Product of x and y will be returned.
+
+by.x
+
+a character vector specifying the joining columns for x.
+
+by.y
+
+a character vector specifying the joining columns for y.
+
+all
+
+a boolean value setting all.x and all.y
+if any of them are unset.
+
+all.x
+
+a boolean value indicating whether all the rows in x should
+be including in the join.
+
+all.y
+
+a boolean value indicating whether all the rows in y should
+be including in the join.
+
+sort
+
+a logical argument indicating whether the resulting columns should be 
sorted.
+
+suffixes
+
+a string vector of length 2 used to make colnames of
+x and y unique.
+The first element is appended to each colname of x.
+The second element is appended to each colname of y.
+
+...
+
+additional argument(s) passed to the method.
+
+
+
+
+Details
+
+If all.x and all.y are set to FALSE, a natural join will be returned. If
+all.x is set to TRUE and all.y is set to FALSE, a left outer join will
+be returned. If all.x is set to FALSE and all.y is set to TRUE, a right
+outer join will be returned. If all.x and all.y are set to TRUE, a full
+outer join will be returned.
+
+
+
+Note
+
+merge since 1.5.0
+
+
+
+See Also
+
+join crossJoin
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,

[56/56] spark-website git commit: Rebuild for 2.2.0

2017-07-11 Thread marmbrus
Rebuild for 2.2.0


Project: http://git-wip-us.apache.org/repos/asf/spark-website/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark-website/commit/ee654d1f
Tree: http://git-wip-us.apache.org/repos/asf/spark-website/tree/ee654d1f
Diff: http://git-wip-us.apache.org/repos/asf/spark-website/diff/ee654d1f

Branch: refs/heads/asf-site
Commit: ee654d1f30dbd724ee21c2b6c0afb46309118882
Parents: 76d1abf
Author: Michael Armbrust 
Authored: Tue Jul 11 19:25:32 2017 +
Committer: Michael Armbrust 
Committed: Tue Jul 11 19:25:32 2017 +

--
 site/committers.html|   8 +-
 site/community.html |   8 +-
 site/contributing.html  |   8 +-
 site/developer-tools.html   |   8 +-
 site/documentation.html |   8 +-
 site/downloads.html |   8 +-
 site/examples.html  |   8 +-
 site/faq.html   |   8 +-
 site/graphx/index.html  |   8 +-
 site/improvement-proposals.html |   8 +-
 site/index.html |   8 +-
 site/js/downloads.js|   1 +
 site/mailing-lists.html |   8 +-
 site/mllib/index.html   |   8 +-
 site/news/amp-camp-2013-registration-ope.html   |   8 +-
 .../news/announcing-the-first-spark-summit.html |   8 +-
 .../news/fourth-spark-screencast-published.html |   8 +-
 site/news/index.html|  17 +-
 site/news/nsdi-paper.html   |   8 +-
 site/news/one-month-to-spark-summit-2015.html   |   8 +-
 .../proposals-open-for-spark-summit-east.html   |   8 +-
 ...registration-open-for-spark-summit-east.html |   8 +-
 .../news/run-spark-and-shark-on-amazon-emr.html |   8 +-
 site/news/spark-0-6-1-and-0-5-2-released.html   |   8 +-
 site/news/spark-0-6-2-released.html |   8 +-
 site/news/spark-0-7-0-released.html |   8 +-
 site/news/spark-0-7-2-released.html |   8 +-
 site/news/spark-0-7-3-released.html |   8 +-
 site/news/spark-0-8-0-released.html |   8 +-
 site/news/spark-0-8-1-released.html |   8 +-
 site/news/spark-0-9-0-released.html |   8 +-
 site/news/spark-0-9-1-released.html |   8 +-
 site/news/spark-0-9-2-released.html |   8 +-
 site/news/spark-1-0-0-released.html |   8 +-
 site/news/spark-1-0-1-released.html |   8 +-
 site/news/spark-1-0-2-released.html |   8 +-
 site/news/spark-1-1-0-released.html |   8 +-
 site/news/spark-1-1-1-released.html |   8 +-
 site/news/spark-1-2-0-released.html |   8 +-
 site/news/spark-1-2-1-released.html |   8 +-
 site/news/spark-1-2-2-released.html |   8 +-
 site/news/spark-1-3-0-released.html |   8 +-
 site/news/spark-1-4-0-released.html |   8 +-
 site/news/spark-1-4-1-released.html |   8 +-
 site/news/spark-1-5-0-released.html |   8 +-
 site/news/spark-1-5-1-released.html |   8 +-
 site/news/spark-1-5-2-released.html |   8 +-
 site/news/spark-1-6-0-released.html |   8 +-
 site/news/spark-1-6-1-released.html |   8 +-
 site/news/spark-1-6-2-released.html |   8 +-
 site/news/spark-1-6-3-released.html |   8 +-
 site/news/spark-2-0-0-released.html |   8 +-
 site/news/spark-2-0-1-released.html |   8 +-
 site/news/spark-2-0-2-released.html |   8 +-
 site/news/spark-2-1-0-released.html |   8 +-
 site/news/spark-2-1-1-released.html |   8 +-
 site/news/spark-2-2-0-released.html | 222 ++
 site/news/spark-2.0.0-preview.html  |   8 +-
 .../spark-accepted-into-apache-incubator.html   |   8 +-
 site/news/spark-and-shark-in-the-news.html  |   8 +-
 site/news/spark-becomes-tlp.html|   8 +-
 site/news/spark-featured-in-wired.html  |   8 +-
 .../spark-mailing-lists-moving-to-apache.html   |   8 +-
 site/news/spark-meetups.html|   8 +-
 site/news/spark-screencasts-published.html  |   8 +-
 site/news/spark-summit-2013-is-a-wrap.html  |   8 +-
 site/news/spark-summit-2014-videos-posted.html  |   8 +-
 site/news/spark-summit-2015-videos-posted.html  |   8 +-
 site/news/spark-summit-agenda-posted.html   |   8 +-
 .../spark-summit-east-2015-videos-posted.html   |   8 +-
 .../spark-summit-east-2016-cfp-closing.html |   8 +-
 .../spark-summit-east-2017-agenda-posted.html   |   8 +-
 site/news/spark-summit-east-agenda-posted.html  |   8 +-
 .../news/spark-summit-europe-agenda-posted.html |   8 +-
 site/news/spark-summit-europe.html  

[54/56] spark-website git commit: Rebuild for 2.2.0

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/releases/spark-release-0-5-2.html
--
diff --git a/site/releases/spark-release-0-5-2.html 
b/site/releases/spark-release-0-5-2.html
index 1ebbb72..ad37cd1 100644
--- a/site/releases/spark-release-0-5-2.html
+++ b/site/releases/spark-release-0-5-2.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/releases/spark-release-0-6-0.html
--
diff --git a/site/releases/spark-release-0-6-0.html 
b/site/releases/spark-release-0-6-0.html
index 65b4904..10e1a69 100644
--- a/site/releases/spark-release-0-6-0.html
+++ b/site/releases/spark-release-0-6-0.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/releases/spark-release-0-6-1.html
--
diff --git a/site/releases/spark-release-0-6-1.html 
b/site/releases/spark-release-0-6-1.html
index d2a22de..ceba185 100644
--- a/site/releases/spark-release-0-6-1.html
+++ b/site/releases/spark-release-0-6-1.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/releases/spark-release-0-6-2.html
--
diff --git a/site/releases/spark-release-0-6-2.html 
b/site/releases/spark-release-0-6-2.html
index 135e3f8..dd39daa 100644
--- a/site/releases/spark-release-0-6-2.html
+++ b/site/releases/spark-release-0-6-2.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released
   (May 02, 2017)
 
@@ -170,9 +173,6 @@
   Spark 
Summit East (Feb 7-9th, 2017, Boston) agenda posted
   (Jan 04, 2017)
 
-  Spark 2.1.0 
released
-  (Dec 28, 2016)
-
   
   Archive
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/ee654d1f/site/releases/spark-release-0-7-0.html
--
diff --git a/site/releases/spark-release-0-7-0.html 
b/site/releases/spark-release-0-7-0.html
index a2e20fb..37cf2ae 100644
--- a/site/releases/spark-release-0-7-0.html
+++ b/site/releases/spark-release-0-7-0.html
@@ -106,7 +106,7 @@
   Documentation 
 
 
-  Latest Release (Spark 2.1.1)
+  Latest Release (Spark 2.2.0)
   Older Versions and Other 
Resources
   Frequently Asked Questions
 
@@ -161,6 +161,9 @@
   Latest News
   
 
+  Spark 2.2.0 
released
+  (Jul 11, 2017)
+
   Spark 2.1.1 
released

[12/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/allclasses-noframe.html
--
diff --git a/site/docs/2.2.0/api/java/allclasses-noframe.html 
b/site/docs/2.2.0/api/java/allclasses-noframe.html
new file mode 100644
index 000..264711a
--- /dev/null
+++ b/site/docs/2.2.0/api/java/allclasses-noframe.html
@@ -0,0 +1,1196 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+All Classes (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+AllClasses
+
+
+AbsoluteError
+Accumulable
+AccumulableInfo
+AccumulableInfo
+AccumulableParam
+Accumulator
+AccumulatorContext
+AccumulatorParam
+AccumulatorParam.DoubleAccumulatorParam$
+AccumulatorParam.FloatAccumulatorParam$
+AccumulatorParam.IntAccumulatorParam$
+AccumulatorParam.LongAccumulatorParam$
+AccumulatorParam.StringAccumulatorParam$
+AccumulatorV2
+AFTAggregator
+AFTCostFun
+AFTSurvivalRegression
+AFTSurvivalRegressionModel
+AggregatedDialect
+AggregatingEdgeContext
+Aggregator
+Aggregator
+Algo
+AllJobsCancelled
+AllReceiverIds
+ALS
+ALS
+ALS.InBlock$
+ALS.Rating
+ALS.Rating$
+ALS.RatingBlock$
+ALSModel
+AnalysisException
+And
+AnyDataType
+ApplicationAttemptInfo
+ApplicationEnvironmentInfo
+ApplicationInfo
+ApplicationsListResource
+ApplicationStatus
+ApplyInPlace
+AreaUnderCurve
+ArrayType
+AskPermissionToCommitOutput
+AssociationRules
+AssociationRules
+AssociationRules.Rule
+AsyncRDDActions
+Attribute
+AttributeGroup
+AttributeKeys
+AttributeType
+BaseRelation
+BaseRRDD
+BasicBlockReplicationPolicy
+BatchInfo
+BatchInfo
+BatchStatus
+BernoulliCellSampler
+BernoulliSampler
+Binarizer
+BinaryAttribute
+BinaryClassificationEvaluator
+BinaryClassificationMetrics
+BinaryLogisticRegressionSummary
+BinaryLogisticRegressionTrainingSummary
+BinarySample
+BinaryType
+BinomialBounds
+BisectingKMeans
+BisectingKMeans
+BisectingKMeansModel
+BisectingKMeansModel
+BisectingKMeansModel.SaveLoadV1_0$
+BisectingKMeansSummary
+BlacklistedExecutor
+BLAS
+BLAS
+BlockId
+BlockManagerId
+BlockManagerMessages
+BlockManagerMessages.BlockManagerHeartbeat
+BlockManagerMessages.BlockManagerHeartbeat$
+BlockManagerMessages.GetBlockStatus
+BlockManagerMessages.GetBlockStatus$
+BlockManagerMessages.GetExecutorEndpointRef
+BlockManagerMessages.GetExecutorEndpointRef$
+BlockManagerMessages.GetLocations
+BlockManagerMessages.GetLocations$
+BlockManagerMessages.GetLocationsMultipleBlockIds
+BlockManagerMessages.GetLocationsMultipleBlockIds$
+BlockManagerMessages.GetMatchingBlockIds
+BlockManagerMessages.GetMatchingBlockIds$
+BlockManagerMessages.GetMemoryStatus$
+BlockManagerMessages.GetPeers
+BlockManagerMessages.GetPeers$
+BlockManagerMessages.GetStorageStatus$
+BlockManagerMessages.HasCachedBlocks
+BlockManagerMessages.HasCachedBlocks$
+BlockManagerMessages.RegisterBlockManager
+BlockManagerMessages.RegisterBlockManager$
+BlockManagerMessages.RemoveBlock
+BlockManagerMessages.RemoveBlock$
+BlockManagerMessages.RemoveBroadcast
+BlockManagerMessages.RemoveBroadcast$
+BlockManagerMessages.RemoveExecutor
+BlockManagerMessages.RemoveExecutor$
+BlockManagerMessages.RemoveRdd
+BlockManagerMessages.RemoveRdd$
+BlockManagerMessages.RemoveShuffle
+BlockManagerMessages.RemoveShuffle$
+BlockManagerMessages.ReplicateBlock
+BlockManagerMessages.ReplicateBlock$
+BlockManagerMessages.StopBlockManagerMaster$
+BlockManagerMessages.ToBlockManagerMaster
+BlockManagerMessages.ToBlockManagerSlave
+BlockManagerMessages.TriggerThreadDump$
+BlockManagerMessages.UpdateBlockInfo
+BlockManagerMessages.UpdateBlockInfo$
+BlockMatrix
+BlockNotFoundException
+BlockReplicationPolicy
+BlockReplicationUtils
+BlockStatus
+BlockUpdatedInfo
+BloomFilter
+BloomFilter.Version
+BooleanParam
+BooleanType
+BoostingStrategy
+BoundedDouble
+BreezeUtil
+Broadcast
+BroadcastBlockId
+Broker
+BucketedRandomProjectionLSH
+BucketedRandomProjectionLSHModel
+Bucketizer
+BufferReleasingInputStream
+BytecodeUtils
+ByteType
+CalendarIntervalType
+Catalog
+CatalystScan
+CategoricalSplit
+CausedBy
+CharType
+CheckpointReader
+CheckpointState
+ChiSqSelector
+ChiSqSelector
+ChiSqSelectorModel
+ChiSqSelectorModel
+ChiSqSelectorModel.SaveLoadV1_0$
+ChiSqTest
+ChiSqTest.Method
+ChiSqTest.Method$
+ChiSqTest.NullHypothesis$
+ChiSqTestResult
+ChiSquareTest
+CholeskyDecomposition
+ClassificationModel
+ClassificationModel
+Classifier
+CleanAccum
+CleanBroadcast
+CleanCheckpoint
+CleanRDD
+CleanShuffle
+CleanupTask
+CleanupTaskWeakReference
+ClosureCleaner
+ClusteringSummary
+CoarseGrainedClusterMessages
+CoarseGrainedClusterMessages.AddWebUIFilter
+CoarseGrainedClusterMessages.AddWebUIFilter$
+CoarseGrainedClusterMessages.GetExecutorLossReason
+CoarseGrainedClusterMessages.GetExecutorLossReason$
+CoarseGrainedClusterMessages.KillExecutors
+CoarseGrainedClusterMessages.KillExecutors$
+CoarseGrainedClusterMessages.KillExecutorsOnHost
+CoarseGrainedClusterMessages.KillExecutorsOnHost$
+CoarseGrainedClusterMessages.KillTask
+CoarseGrainedClusterMessages.KillTask$

[47/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/add_months.html
--
diff --git a/site/docs/2.2.0/api/R/add_months.html 
b/site/docs/2.2.0/api/R/add_months.html
new file mode 100644
index 000..b2f2915
--- /dev/null
+++ b/site/docs/2.2.0/api/R/add_months.html
@@ -0,0 +1,122 @@
+
+R: add_months
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+add_months 
{SparkR}R Documentation
+
+add_months
+
+Description
+
+Returns the date that is numMonths after startDate.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column,numeric'
+add_months(y, x)
+
+add_months(y, x)
+
+
+
+Arguments
+
+
+y
+
+Column to compute on
+
+x
+
+Number of months to add
+
+
+
+
+Note
+
+add_months since 1.5.0
+
+
+
+See Also
+
+Other datetime_funcs: date_add,
+date_add,
+date_add,Column,numeric-method;
+date_format, date_format,
+date_format,Column,character-method;
+date_sub, date_sub,
+date_sub,Column,numeric-method;
+datediff, datediff,
+datediff,Column-method;
+dayofmonth, dayofmonth,
+dayofmonth,Column-method;
+dayofyear, dayofyear,
+dayofyear,Column-method;
+from_unixtime, from_unixtime,
+from_unixtime,Column-method;
+from_utc_timestamp,
+from_utc_timestamp,
+from_utc_timestamp,Column,character-method;
+hour, hour,
+hour,Column-method; last_day,
+last_day,
+last_day,Column-method;
+minute, minute,
+minute,Column-method;
+months_between,
+months_between,
+months_between,Column-method;
+month, month,
+month,Column-method;
+next_day, next_day,
+next_day,Column,character-method;
+quarter, quarter,
+quarter,Column-method;
+second, second,
+second,Column-method;
+to_date, to_date,
+to_date,
+to_date,Column,character-method,
+to_date,Column,missing-method;
+to_timestamp, to_timestamp,
+to_timestamp,
+to_timestamp,Column,character-method,
+to_timestamp,Column,missing-method;
+to_utc_timestamp,
+to_utc_timestamp,
+to_utc_timestamp,Column,character-method;
+unix_timestamp,
+unix_timestamp,
+unix_timestamp,
+unix_timestamp,
+unix_timestamp,Column,character-method,
+unix_timestamp,Column,missing-method,
+unix_timestamp,missing,missing-method;
+weekofyear, weekofyear,
+weekofyear,Column-method;
+window, window,
+window,Column-method; year,
+year, year,Column-method
+
+
+
+Examples
+
+## Not run: add_months(df$d, 1)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/alias.html
--
diff --git a/site/docs/2.2.0/api/R/alias.html b/site/docs/2.2.0/api/R/alias.html
new file mode 100644
index 000..61606dc
--- /dev/null
+++ b/site/docs/2.2.0/api/R/alias.html
@@ -0,0 +1,63 @@
+
+R: alias
+
+
+
+
+alias 
{SparkR}R Documentation
+
+alias
+
+Description
+
+Set a new name for a column
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+alias(object, data)
+
+
+
+Arguments
+
+
+object
+
+Column to rename
+
+data
+
+new name to use
+
+
+
+
+Note
+
+alias since 1.4.0
+
+
+
+See Also
+
+Other colum_func: between,
+between,
+between,Column-method; cast,
+cast, cast,Column-method;
+endsWith, endsWith,
+endsWith,Column-method;
+otherwise, otherwise,
+otherwise,Column-method;
+over, over,
+over,Column,WindowSpec-method;
+startsWith, startsWith,
+startsWith,Column-method;
+substr, substr,Column-method
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/approxCountDistinct.html
--
diff --git a/site/docs/2.2.0/api/R/approxCountDistinct.html 
b/site/docs/2.2.0/api/R/approxCountDistinct.html
new file mode 100644
index 000..a655bc2
--- /dev/null
+++ b/site/docs/2.2.0/api/R/approxCountDistinct.html
@@ -0,0 +1,76 @@
+
+R: Returns the approximate number of distinct items in a 
group
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+approxCountDistinct {SparkR}R 
Documentation
+
+Returns the approximate number of distinct items in a group
+
+Description
+
+Returns the approximate number of distinct items in a group. This is a 
column
+aggregate function.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+approxCountDistinct(x, rsd = 0.05)
+
+## S4 method for signature 'Column'
+approxCountDistinct(x, rsd = 0.05)
+
+approxCountDistinct(x, ...)
+
+
+
+Arguments
+
+
+x
+
+Column to compute on.
+
+rsd
+
+maximum estimation error allowed (default = 0.05)
+
+...
+
+further 

[04/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/CleanShuffle.html
--
diff --git a/site/docs/2.2.0/api/java/org/apache/spark/CleanShuffle.html 
b/site/docs/2.2.0/api/java/org/apache/spark/CleanShuffle.html
new file mode 100644
index 000..ecc92d2
--- /dev/null
+++ b/site/docs/2.2.0/api/java/org/apache/spark/CleanShuffle.html
@@ -0,0 +1,370 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+CleanShuffle (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+var methods = {"i0":5,"i1":5,"i2":5,"i3":5,"i4":9,"i5":9,"i6":10};
+var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],4:["t3","Abstract 
Methods"],8:["t4","Concrete Methods"]};
+var altColor = "altColor";
+var rowColor = "rowColor";
+var tableTab = "tableTab";
+var activeTableTab = "activeTableTab";
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+
+org.apache.spark
+Class CleanShuffle
+
+
+
+Object
+
+
+org.apache.spark.CleanShuffle
+
+
+
+
+
+
+
+All Implemented Interfaces:
+java.io.Serializable, CleanupTask, scala.Equals, 
scala.Product
+
+
+
+public class CleanShuffle
+extends Object
+implements CleanupTask, scala.Product, 
scala.Serializable
+
+See Also:
+Serialized 
Form
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors
+
+Constructor and Description
+
+
+CleanShuffle(intshuffleId)
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All MethodsStatic MethodsInstance MethodsAbstract MethodsConcrete Methods
+
+Modifier and Type
+Method and Description
+
+
+abstract static boolean
+canEqual(Objectthat)
+
+
+abstract static boolean
+equals(Objectthat)
+
+
+abstract static int
+productArity()
+
+
+abstract static Object
+productElement(intn)
+
+
+static 
scala.collection.IteratorObject
+productIterator()
+
+
+static String
+productPrefix()
+
+
+int
+shuffleId()
+
+
+
+
+
+
+Methods inherited from classObject
+equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, 
wait
+
+
+
+
+
+Methods inherited from interfacescala.Product
+productArity, productElement, productIterator, productPrefix
+
+
+
+
+
+Methods inherited from interfacescala.Equals
+canEqual, equals
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Detail
+
+
+
+
+
+CleanShuffle
+publicCleanShuffle(intshuffleId)
+
+
+
+
+
+
+
+
+
+Method Detail
+
+
+
+
+
+canEqual
+public abstract staticbooleancanEqual(Objectthat)
+
+
+
+
+
+
+
+equals
+public abstract staticbooleanequals(Objectthat)
+
+
+
+
+
+
+
+productElement
+public abstract staticObjectproductElement(intn)
+
+
+
+
+
+
+
+productArity
+public abstract staticintproductArity()
+
+
+
+
+
+
+
+productIterator
+public 
staticscala.collection.IteratorObjectproductIterator()
+
+
+
+
+
+
+
+productPrefix
+public staticStringproductPrefix()
+
+
+
+
+
+
+
+shuffleId
+publicintshuffleId()
+
+
+
+
+
+
+
+
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/CleanupTask.html
--
diff --git a/site/docs/2.2.0/api/java/org/apache/spark/CleanupTask.html 
b/site/docs/2.2.0/api/java/org/apache/spark/CleanupTask.html
new file mode 100644
index 000..3f007cf
--- /dev/null
+++ b/site/docs/2.2.0/api/java/org/apache/spark/CleanupTask.html
@@ -0,0 +1,170 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+CleanupTask (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+

[43/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/cosh.html
--
diff --git a/site/docs/2.2.0/api/R/cosh.html b/site/docs/2.2.0/api/R/cosh.html
new file mode 100644
index 000..0b06194
--- /dev/null
+++ b/site/docs/2.2.0/api/R/cosh.html
@@ -0,0 +1,120 @@
+
+R: cosh
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+cosh 
{SparkR}R Documentation
+
+cosh
+
+Description
+
+Computes the hyperbolic cosine of the given value.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+cosh(x)
+
+
+
+Arguments
+
+
+x
+
+Column to compute on.
+
+
+
+
+Note
+
+cosh since 1.5.0
+
+
+
+See Also
+
+Other math_funcs: acos,
+acos,Column-method; asin,
+asin,Column-method; atan2,
+atan2,Column-method; atan,
+atan,Column-method; bin,
+bin, bin,Column-method;
+bround, bround,
+bround,Column-method; cbrt,
+cbrt, cbrt,Column-method;
+ceil, ceil,
+ceil,Column-method, ceiling,
+ceiling,Column-method; conv,
+conv,
+conv,Column,numeric,numeric-method;
+corr, corr,
+corr, corr,Column-method,
+corr,SparkDataFrame-method;
+cos, cos,Column-method;
+covar_pop, covar_pop,
+covar_pop,characterOrColumn,characterOrColumn-method;
+cov, cov, cov,
+cov,SparkDataFrame-method,
+cov,characterOrColumn-method,
+covar_samp, covar_samp,
+covar_samp,characterOrColumn,characterOrColumn-method;
+expm1, expm1,Column-method;
+exp, exp,Column-method;
+factorial,
+factorial,Column-method;
+floor, floor,Column-method;
+hex, hex,
+hex,Column-method; hypot,
+hypot, hypot,Column-method;
+log10, log10,Column-method;
+log1p, log1p,Column-method;
+log2, log2,Column-method;
+log, log,Column-method;
+pmod, pmod,
+pmod,Column-method; rint,
+rint, rint,Column-method;
+round, round,Column-method;
+shiftLeft, shiftLeft,
+shiftLeft,Column,numeric-method;
+shiftRightUnsigned,
+shiftRightUnsigned,
+shiftRightUnsigned,Column,numeric-method;
+shiftRight, shiftRight,
+shiftRight,Column,numeric-method;
+sign, sign,Column-method,
+signum, signum,
+signum,Column-method; sinh,
+sinh,Column-method; sin,
+sin,Column-method; sqrt,
+sqrt,Column-method; tanh,
+tanh,Column-method; tan,
+tan,Column-method; toDegrees,
+toDegrees,
+toDegrees,Column-method;
+toRadians, toRadians,
+toRadians,Column-method;
+unhex, unhex,
+unhex,Column-method
+
+
+
+Examples
+
+## Not run: cosh(df$c)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/count.html
--
diff --git a/site/docs/2.2.0/api/R/count.html b/site/docs/2.2.0/api/R/count.html
new file mode 100644
index 000..5ee31e8
--- /dev/null
+++ b/site/docs/2.2.0/api/R/count.html
@@ -0,0 +1,129 @@
+
+R: Returns the number of items in a group
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+count 
{SparkR}R Documentation
+
+Returns the number of items in a group
+
+Description
+
+This can be used as a column aggregate function with Column as 
input,
+and returns the number of items in a group.
+
+Count the number of rows for each group when we have 
GroupedData input.
+The resulting SparkDataFrame will also contain the grouping columns.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+count(x)
+
+## S4 method for signature 'Column'
+n(x)
+
+count(x)
+
+n(x)
+
+## S4 method for signature 'GroupedData'
+count(x)
+
+
+
+Arguments
+
+
+x
+
+a GroupedData or Column.
+
+
+
+
+Value
+
+A SparkDataFrame.
+
+
+
+Note
+
+count since 1.4.0
+
+n since 1.4.0
+
+count since 1.4.0
+
+
+
+See Also
+
+Other agg_funcs: agg, agg,
+agg, agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+avg, avg,
+avg,Column-method;
+countDistinct, countDistinct,
+countDistinct,Column-method,
+n_distinct, n_distinct,
+n_distinct,Column-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+kurtosis, kurtosis,
+kurtosis,Column-method; last,
+last,
+last,characterOrColumn-method;
+max, max,Column-method;
+mean, mean,Column-method;
+min, min,Column-method;
+sd, sd,
+sd,Column-method, stddev,
+stddev, stddev,Column-method;
+skewness, skewness,
+skewness,Column-method;
+stddev_pop, stddev_pop,
+stddev_pop,Column-method;
+stddev_samp, stddev_samp,
+stddev_samp,Column-method;
+sumDistinct, sumDistinct,
+sumDistinct,Column-method;
+sum, sum,Column-method;
+var_pop, var_pop,
+var_pop,Column-method;
+var_samp, 

[29/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/over.html
--
diff --git a/site/docs/2.2.0/api/R/over.html b/site/docs/2.2.0/api/R/over.html
new file mode 100644
index 000..539a56f
--- /dev/null
+++ b/site/docs/2.2.0/api/R/over.html
@@ -0,0 +1,88 @@
+
+R: over
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+over 
{SparkR}R Documentation
+
+over
+
+Description
+
+Define a windowing column.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column,WindowSpec'
+over(x, window)
+
+over(x, window)
+
+
+
+Arguments
+
+
+x
+
+a Column, usually one returned by window function(s).
+
+window
+
+a WindowSpec object. Can be created by windowPartitionBy or
+windowOrderBy and configured by other WindowSpec methods.
+
+
+
+
+Note
+
+over since 2.0.0
+
+
+
+See Also
+
+Other colum_func: alias,
+alias,Column-method; between,
+between,
+between,Column-method; cast,
+cast, cast,Column-method;
+endsWith, endsWith,
+endsWith,Column-method;
+otherwise, otherwise,
+otherwise,Column-method;
+startsWith, startsWith,
+startsWith,Column-method;
+substr, substr,Column-method
+
+
+
+Examples
+
+## Not run: 
+##D   df - createDataFrame(mtcars)
+##D 
+##D   # Partition by am (transmission) and order by hp (horsepower)
+##D   ws - orderBy(windowPartitionBy(am), hp)
+##D 
+##D   # Rank on hp within each partition
+##D   out - select(df, over(rank(), ws), df$hp, df$am)
+##D 
+##D   # Lag mpg values by 1 row on the partition-and-ordered table
+##D   out - select(df, over(lead(df$mpg), ws), df$mpg, df$hp, df$am)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/partitionBy.html
--
diff --git a/site/docs/2.2.0/api/R/partitionBy.html 
b/site/docs/2.2.0/api/R/partitionBy.html
new file mode 100644
index 000..ae33652
--- /dev/null
+++ b/site/docs/2.2.0/api/R/partitionBy.html
@@ -0,0 +1,85 @@
+
+R: partitionBy
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+partitionBy {SparkR}R 
Documentation
+
+partitionBy
+
+Description
+
+Defines the partitioning columns in a WindowSpec.
+
+
+
+Usage
+
+
+## S4 method for signature 'WindowSpec'
+partitionBy(x, col, ...)
+
+partitionBy(x, ...)
+
+
+
+Arguments
+
+
+x
+
+a WindowSpec.
+
+col
+
+a column to partition on (desribed by the name or Column).
+
+...
+
+additional column(s) to partition on.
+
+
+
+
+Value
+
+A WindowSpec.
+
+
+
+Note
+
+partitionBy(WindowSpec) since 2.0.0
+
+
+
+See Also
+
+Other windowspec_method: orderBy,
+orderBy, orderBy,
+orderBy,WindowSpec,Column-method,
+orderBy,WindowSpec,character-method;
+rangeBetween, rangeBetween,
+rangeBetween,WindowSpec,numeric,numeric-method;
+rowsBetween, rowsBetween,
+rowsBetween,WindowSpec,numeric,numeric-method
+
+
+
+Examples
+
+## Not run: 
+##D   partitionBy(ws, col1, col2)
+##D   partitionBy(ws, df$col1, df$col2)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/percent_rank.html
--
diff --git a/site/docs/2.2.0/api/R/percent_rank.html 
b/site/docs/2.2.0/api/R/percent_rank.html
new file mode 100644
index 000..2f9dae4
--- /dev/null
+++ b/site/docs/2.2.0/api/R/percent_rank.html
@@ -0,0 +1,90 @@
+
+R: percent_rank
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+percent_rank {SparkR}R 
Documentation
+
+percent_rank
+
+Description
+
+Window function: returns the relative rank (i.e. percentile) of rows within 
a window partition.
+
+
+
+Usage
+
+
+## S4 method for signature 'missing'
+percent_rank()
+
+percent_rank(x = "missing")
+
+
+
+Arguments
+
+
+x
+
+empty. Should be used with no argument.
+
+
+
+
+Details
+
+This is computed by:
+
+(rank of row in its partition - 1) / (number of rows in the partition - 1)
+
+This is equivalent to the PERCENT_RANK function in SQL.
+
+
+
+Note
+
+percent_rank since 1.6.0
+
+
+
+See Also
+
+Other window_funcs: cume_dist,
+cume_dist,
+cume_dist,missing-method;
+dense_rank, dense_rank,
+dense_rank,missing-method;
+lag, lag,

[17/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/unpersist.html
--
diff --git a/site/docs/2.2.0/api/R/unpersist.html 
b/site/docs/2.2.0/api/R/unpersist.html
new file mode 100644
index 000..b514546
--- /dev/null
+++ b/site/docs/2.2.0/api/R/unpersist.html
@@ -0,0 +1,294 @@
+
+R: Unpersist
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+unpersist 
{SparkR}R Documentation
+
+Unpersist
+
+Description
+
+Mark this SparkDataFrame as non-persistent, and remove all blocks for it 
from memory and
+disk.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+unpersist(x, blocking = TRUE)
+
+unpersist(x, ...)
+
+
+
+Arguments
+
+
+x
+
+the SparkDataFrame to unpersist.
+
+blocking
+
+whether to block until all blocks are deleted.
+
+...
+
+further arguments to be passed to or from other methods.
+
+
+
+
+Note
+
+unpersist since 1.4.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+intersect, intersect,
+intersect,SparkDataFrame,SparkDataFrame-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,
+join,SparkDataFrame,SparkDataFrame-method;
+limit, limit,
+limit,SparkDataFrame,numeric-method;
+merge, merge,

[35/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/histogram.html
--
diff --git a/site/docs/2.2.0/api/R/histogram.html 
b/site/docs/2.2.0/api/R/histogram.html
new file mode 100644
index 000..417fb38
--- /dev/null
+++ b/site/docs/2.2.0/api/R/histogram.html
@@ -0,0 +1,306 @@
+
+R: Compute histogram statistics for given column
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+histogram 
{SparkR}R Documentation
+
+Compute histogram statistics for given column
+
+Description
+
+This function computes a histogram for a given SparkR Column.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,characterOrColumn'
+histogram(df, col, nbins = 10)
+
+
+
+Arguments
+
+
+df
+
+the SparkDataFrame containing the Column to build the histogram from.
+
+col
+
+the column as Character string or a Column to build the histogram from.
+
+nbins
+
+the number of bins (optional). Default value is 10.
+
+
+
+
+Value
+
+a data.frame with the histogram statistics, i.e., counts and centroids.
+
+
+
+Note
+
+histogram since 2.0.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+intersect, intersect,
+intersect,SparkDataFrame,SparkDataFrame-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,

[16/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/with.html
--
diff --git a/site/docs/2.2.0/api/R/with.html b/site/docs/2.2.0/api/R/with.html
new file mode 100644
index 000..d6e656c
--- /dev/null
+++ b/site/docs/2.2.0/api/R/with.html
@@ -0,0 +1,295 @@
+
+R: Evaluate a R expression in an environment constructed 
from a...
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+with,SparkDataFrame-method {SparkR}R 
Documentation
+
+Evaluate a R expression in an environment constructed from a 
SparkDataFrame
+
+Description
+
+Evaluate a R expression in an environment constructed from a SparkDataFrame
+with() allows access to columns of a SparkDataFrame by simply referring to
+their name. It appends every column of a SparkDataFrame into a new
+environment. Then, the given expression is evaluated in this new
+environment.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+with(data, expr, ...)
+
+with(data, expr, ...)
+
+
+
+Arguments
+
+
+data
+
+(SparkDataFrame) SparkDataFrame to use for constructing an environment.
+
+expr
+
+(expression) Expression to evaluate.
+
+...
+
+arguments to be passed to future methods.
+
+
+
+
+Note
+
+with since 1.6.0
+
+
+
+See Also
+
+attach
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,

[10/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/index-all.html
--
diff --git a/site/docs/2.2.0/api/java/index-all.html 
b/site/docs/2.2.0/api/java/index-all.html
new file mode 100644
index 000..9577386
--- /dev/null
+++ b/site/docs/2.2.0/api/java/index-all.html
@@ -0,0 +1,48898 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+Index (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+Prev
+Next
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+
+
+$ABCDEFGHIJKLMNOPQRSTUVWXYZ_
+
+
+$
+
+$colon$bslash(B,
 Function2A, B, B) - Static method in class 
org.apache.spark.sql.types.StructType
+
+$colon$plus(B,
 CanBuildFromRepr, B, That) - Static method in class 
org.apache.spark.sql.types.StructType
+
+$div$colon(B,
 Function2B, A, B) - Static method in class 
org.apache.spark.sql.types.StructType
+
+$greater(A)
 - Static method in class org.apache.spark.sql.types.Decimal
+
+$greater(A)
 - Static method in class org.apache.spark.storage.RDDInfo
+
+$greater$eq(A)
 - Static method in class org.apache.spark.sql.types.Decimal
+
+$greater$eq(A)
 - Static method in class org.apache.spark.storage.RDDInfo
+
+$less(A) 
- Static method in class org.apache.spark.sql.types.Decimal
+
+$less(A) - 
Static method in class org.apache.spark.storage.RDDInfo
+
+$less$eq(A)
 - Static method in class org.apache.spark.sql.types.Decimal
+
+$less$eq(A)
 - Static method in class org.apache.spark.storage.RDDInfo
+
+$minus$greater(T)
 - Static method in class org.apache.spark.ml.param.DoubleParam
+
+$minus$greater(T)
 - Static method in class org.apache.spark.ml.param.FloatParam
+
+$plus$colon(B,
 CanBuildFromRepr, B, That) - Static method in class 
org.apache.spark.sql.types.StructType
+
+$plus$eq(T)
 - Static method in class org.apache.spark.Accumulator
+
+Deprecated.
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.api.r.RRDD
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.graphx.EdgeRDD
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.graphx.impl.EdgeRDDImpl
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.graphx.impl.VertexRDDImpl
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.graphx.VertexRDD
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.rdd.HadoopRDD
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.rdd.JdbcRDD
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.rdd.NewHadoopRDD
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.rdd.PartitionPruningRDD
+
+$plus$plus(RDDT)
 - Static method in class org.apache.spark.rdd.UnionRDD
+
+$plus$plus(GenTraversableOnceB,
 CanBuildFromRepr, B, That) - Static method in class 
org.apache.spark.sql.types.StructType
+
+$plus$plus$colon(TraversableOnceB,
 CanBuildFromRepr, B, That) - Static method in class 
org.apache.spark.sql.types.StructType
+
+$plus$plus$colon(TraversableB,
 CanBuildFromRepr, B, That) - Static method in class 
org.apache.spark.sql.types.StructType
+
+$plus$plus$eq(R)
 - Static method in class org.apache.spark.Accumulator
+
+Deprecated.
+
+
+
+
+
+A
+
+abortJob(JobContext)
 - Method in class org.apache.spark.internal.io.FileCommitProtocol
+
+Aborts a job after the writes fail.
+
+abortJob(JobContext)
 - Method in class org.apache.spark.internal.io.HadoopMapReduceCommitProtocol
+
+abortTask(TaskAttemptContext)
 - Method in class org.apache.spark.internal.io.FileCommitProtocol
+
+Aborts a task after the writes have failed.
+
+abortTask(TaskAttemptContext)
 - Method in class org.apache.spark.internal.io.HadoopMapReduceCommitProtocol
+
+abs(Column)
 - Static method in class org.apache.spark.sql.functions
+
+Computes the absolute value.
+
+abs() - Method 
in class org.apache.spark.sql.types.Decimal
+
+absent() - 
Static method in class org.apache.spark.api.java.Optional
+
+AbsoluteError - Class in org.apache.spark.mllib.tree.loss
+
+:: DeveloperApi ::
+ Class for absolute error loss calculation (for regression).
+
+AbsoluteError()
 - Constructor for class org.apache.spark.mllib.tree.loss.AbsoluteError
+
+accept(Parsers)
 - Static method in class org.apache.spark.ml.feature.RFormulaParser
+
+accept(ES,
 Function1ES, ListObject) - Static method in class 
org.apache.spark.ml.feature.RFormulaParser
+
+accept(String,
 PartialFunctionObject, U) - Static method in class 
org.apache.spark.ml.feature.RFormulaParser
+

[15/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/write.json.html
--
diff --git a/site/docs/2.2.0/api/R/write.json.html 
b/site/docs/2.2.0/api/R/write.json.html
new file mode 100644
index 000..c7eada3
--- /dev/null
+++ b/site/docs/2.2.0/api/R/write.json.html
@@ -0,0 +1,298 @@
+
+R: Save the contents of SparkDataFrame as a JSON 
file
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+write.json 
{SparkR}R Documentation
+
+Save the contents of SparkDataFrame as a JSON file
+
+Description
+
+Save the contents of a SparkDataFrame as a JSON file (http://jsonlines.org/;>
+JSON Lines text format or newline-delimited JSON). Files written out
+with this method can be read back in as a SparkDataFrame using read.json().
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,character'
+write.json(x, path, mode = "error", ...)
+
+write.json(x, path, ...)
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame
+
+path
+
+The directory where the file is saved
+
+mode
+
+one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by 
default)
+
+...
+
+additional argument(s) passed to the method.
+
+
+
+
+Note
+
+write.json since 1.6.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;

[46/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/atan2.html
--
diff --git a/site/docs/2.2.0/api/R/atan2.html b/site/docs/2.2.0/api/R/atan2.html
new file mode 100644
index 000..4a141c7
--- /dev/null
+++ b/site/docs/2.2.0/api/R/atan2.html
@@ -0,0 +1,125 @@
+
+R: atan2
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+atan2 
{SparkR}R Documentation
+
+atan2
+
+Description
+
+Returns the angle theta from the conversion of rectangular coordinates (x, 
y) to
+polar coordinates (r, theta).
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+atan2(y, x)
+
+
+
+Arguments
+
+
+y
+
+Column to compute on.
+
+x
+
+Column to compute on.
+
+
+
+
+Note
+
+atan2 since 1.5.0
+
+
+
+See Also
+
+Other math_funcs: acos,
+acos,Column-method; asin,
+asin,Column-method; atan,
+atan,Column-method; bin,
+bin, bin,Column-method;
+bround, bround,
+bround,Column-method; cbrt,
+cbrt, cbrt,Column-method;
+ceil, ceil,
+ceil,Column-method, ceiling,
+ceiling,Column-method; conv,
+conv,
+conv,Column,numeric,numeric-method;
+corr, corr,
+corr, corr,Column-method,
+corr,SparkDataFrame-method;
+cosh, cosh,Column-method;
+cos, cos,Column-method;
+covar_pop, covar_pop,
+covar_pop,characterOrColumn,characterOrColumn-method;
+cov, cov, cov,
+cov,SparkDataFrame-method,
+cov,characterOrColumn-method,
+covar_samp, covar_samp,
+covar_samp,characterOrColumn,characterOrColumn-method;
+expm1, expm1,Column-method;
+exp, exp,Column-method;
+factorial,
+factorial,Column-method;
+floor, floor,Column-method;
+hex, hex,
+hex,Column-method; hypot,
+hypot, hypot,Column-method;
+log10, log10,Column-method;
+log1p, log1p,Column-method;
+log2, log2,Column-method;
+log, log,Column-method;
+pmod, pmod,
+pmod,Column-method; rint,
+rint, rint,Column-method;
+round, round,Column-method;
+shiftLeft, shiftLeft,
+shiftLeft,Column,numeric-method;
+shiftRightUnsigned,
+shiftRightUnsigned,
+shiftRightUnsigned,Column,numeric-method;
+shiftRight, shiftRight,
+shiftRight,Column,numeric-method;
+sign, sign,Column-method,
+signum, signum,
+signum,Column-method; sinh,
+sinh,Column-method; sin,
+sin,Column-method; sqrt,
+sqrt,Column-method; tanh,
+tanh,Column-method; tan,
+tan,Column-method; toDegrees,
+toDegrees,
+toDegrees,Column-method;
+toRadians, toRadians,
+toRadians,Column-method;
+unhex, unhex,
+unhex,Column-method
+
+
+
+Examples
+
+## Not run: atan2(df$c, x)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/attach.html
--
diff --git a/site/docs/2.2.0/api/R/attach.html 
b/site/docs/2.2.0/api/R/attach.html
new file mode 100644
index 000..6613f8c
--- /dev/null
+++ b/site/docs/2.2.0/api/R/attach.html
@@ -0,0 +1,302 @@
+
+R: Attach SparkDataFrame to R search path
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+attach,SparkDataFrame-method {SparkR}R 
Documentation
+
+Attach SparkDataFrame to R search path
+
+Description
+
+The specified SparkDataFrame is attached to the R search path. This means 
that
+the SparkDataFrame is searched by R when evaluating a variable, so columns in
+the SparkDataFrame can be accessed by simply giving their names.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+attach(what, pos = 2,
+  name = deparse(substitute(what)), warn.conflicts = TRUE)
+
+attach(what, pos = 2L, name = deparse(substitute(what)),
+  warn.conflicts = TRUE)
+
+
+
+Arguments
+
+
+what
+
+(SparkDataFrame) The SparkDataFrame to attach
+
+pos
+
+(integer) Specify position in search() where to attach.
+
+name
+
+(character) Name to use for the attached SparkDataFrame. Names
+starting with package: are reserved for library.
+
+warn.conflicts
+
+(logical) If TRUE, warnings are printed about conflicts
+from attaching the database, unless that SparkDataFrame contains an object
+
+
+
+
+Note
+
+attach since 1.6.0
+
+
+
+See Also
+
+detach
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,

[25/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/schema.html
--
diff --git a/site/docs/2.2.0/api/R/schema.html 
b/site/docs/2.2.0/api/R/schema.html
new file mode 100644
index 000..e4112c8
--- /dev/null
+++ b/site/docs/2.2.0/api/R/schema.html
@@ -0,0 +1,284 @@
+
+R: Get schema object
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+schema 
{SparkR}R Documentation
+
+Get schema object
+
+Description
+
+Returns the schema of this SparkDataFrame as a structType object.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame'
+schema(x)
+
+schema(x)
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame
+
+
+
+
+Note
+
+schema since 1.4.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,
+insertInto,SparkDataFrame,character-method;
+intersect, intersect,
+intersect,SparkDataFrame,SparkDataFrame-method;
+isLocal, isLocal,
+isLocal,SparkDataFrame-method;
+isStreaming, isStreaming,
+isStreaming,SparkDataFrame-method;
+join,
+join,SparkDataFrame,SparkDataFrame-method;
+limit, limit,
+limit,SparkDataFrame,numeric-method;
+merge, merge,
+merge,SparkDataFrame,SparkDataFrame-method;
+mutate, mutate,
+mutate,SparkDataFrame-method,
+transform, transform,
+transform,SparkDataFrame-method;
+ncol,
+ncol,SparkDataFrame-method;
+persist, persist,

[52/56] spark-website git commit: Update for 2.2.0

2017-07-11 Thread marmbrus
Update for 2.2.0


Project: http://git-wip-us.apache.org/repos/asf/spark-website/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark-website/commit/76d1abf1
Tree: http://git-wip-us.apache.org/repos/asf/spark-website/tree/76d1abf1
Diff: http://git-wip-us.apache.org/repos/asf/spark-website/diff/76d1abf1

Branch: refs/heads/asf-site
Commit: 76d1abf1fc7be6d4455958b0ce3e35b240a21342
Parents: f7ec115
Author: Michael Armbrust 
Authored: Mon Jul 10 16:48:58 2017 -0700
Committer: Michael Armbrust 
Committed: Tue Jul 11 19:24:06 2017 +

--
 _layouts/global.html|   2 +-
 js/downloads.js |   1 +
 news/_posts/2017-07-11-spark-2-2-0-released.md  |  14 ++
 .../_posts/2017-07-11-spark-release-2-2-0.md| 151 +++
 site/docs/latest|   2 +-
 5 files changed, 168 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark-website/blob/76d1abf1/_layouts/global.html
--
diff --git a/_layouts/global.html b/_layouts/global.html
index bcd83ad..778a841 100644
--- a/_layouts/global.html
+++ b/_layouts/global.html
@@ -121,7 +121,7 @@
   Documentation 
 
 
-  Latest Release (Spark 
2.1.1)
+  Latest Release (Spark 
2.2.0)
   Older Versions and 
Other Resources
   Frequently Asked 
Questions
 

http://git-wip-us.apache.org/repos/asf/spark-website/blob/76d1abf1/js/downloads.js
--
diff --git a/js/downloads.js b/js/downloads.js
index d308389..d2b43a7 100644
--- a/js/downloads.js
+++ b/js/downloads.js
@@ -28,6 +28,7 @@ var packagesV8 = [hadoop2p7, hadoop2p6, hadoopFree, sources]
 
 //addRelease("2.2.0", new Date("x/x/2017"), packagesV8, true);
 
+addRelease("2.2.0", new Date("07/11/2017"), packagesV8, true);
 addRelease("2.1.1", new Date("05/02/2017"), packagesV7, true);
 addRelease("2.1.0", new Date("12/28/2016"), packagesV7, true);
 addRelease("2.0.2", new Date("11/14/2016"), packagesV7, true);

http://git-wip-us.apache.org/repos/asf/spark-website/blob/76d1abf1/news/_posts/2017-07-11-spark-2-2-0-released.md
--
diff --git a/news/_posts/2017-07-11-spark-2-2-0-released.md 
b/news/_posts/2017-07-11-spark-2-2-0-released.md
new file mode 100644
index 000..6cb11f1
--- /dev/null
+++ b/news/_posts/2017-07-11-spark-2-2-0-released.md
@@ -0,0 +1,14 @@
+---
+layout: post
+title: Spark 2.2.0 released
+categories:
+- News
+tags: []
+status: publish
+type: post
+published: true
+meta:
+  _edit_last: '4'
+  _wpas_done_all: '1'
+---
+We are happy to announce the availability of Spark 2.2.0! Visit the release notes to read about the new features, or download the release today.

http://git-wip-us.apache.org/repos/asf/spark-website/blob/76d1abf1/releases/_posts/2017-07-11-spark-release-2-2-0.md
--
diff --git a/releases/_posts/2017-07-11-spark-release-2-2-0.md 
b/releases/_posts/2017-07-11-spark-release-2-2-0.md
new file mode 100644
index 000..8027d8a
--- /dev/null
+++ b/releases/_posts/2017-07-11-spark-release-2-2-0.md
@@ -0,0 +1,151 @@
+---
+layout: post
+title: Spark Release 2.2.0
+categories: []
+tags: []
+status: publish
+type: post
+published: true
+meta:
+  _edit_last: '4'
+  _wpas_done_all: '1'
+---
+
+
+Apache Spark 2.2.0 is the third release on the 2.x line. This release removes 
the experimental tag from Structured Streaming. In addition, this release 
focuses more on usability, stability, and polish, resolving over 1100 tickets.
+
+
+To download Apache Spark 2.2.0, visit the downloads page. You can consult JIRA 
for the [detailed 
changes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315420=12338275).
 We have curated a list of high level changes here, grouped by major modules.
+
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+
+### Core and Spark SQL
+
+ - **API updates**
+   - SPARK-19107: Support creating hive table with DataFrameWriter and Catalog
+   - SPARK-13721: Add support for LATERAL VIEW OUTER explode()
+   - SPARK-18885: Unify CREATE TABLE syntax for data source and hive serde 
tables
+   - SPARK-16475: Added Broadcast Hints BROADCAST, BROADCASTJOIN, and MAPJOIN, 
 for SQL Queries
+   - SPARK-18350: Support session local timezone
+   - SPARK-19261: Support ALTER TABLE table_name ADD COLUMNS
+   - SPARK-20420: Add events to the external catalog
+   - SPARK-18127: Add hooks and extension points to Spark
+   - SPARK-20576: Support generic hint function in Dataset/DataFrame
+   - SPARK-17203: Data source options should always be 

[14/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/write.text.html
--
diff --git a/site/docs/2.2.0/api/R/write.text.html 
b/site/docs/2.2.0/api/R/write.text.html
new file mode 100644
index 000..e422215
--- /dev/null
+++ b/site/docs/2.2.0/api/R/write.text.html
@@ -0,0 +1,298 @@
+
+R: Save the content of SparkDataFrame in a text file at 
the...
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+write.text 
{SparkR}R Documentation
+
+Save the content of SparkDataFrame in a text file at the specified 
path.
+
+Description
+
+Save the content of the SparkDataFrame in a text file at the specified path.
+The SparkDataFrame must have only one column of string type with the name 
value.
+Each row becomes a new line in the output file.
+
+
+
+Usage
+
+
+## S4 method for signature 'SparkDataFrame,character'
+write.text(x, path, mode = "error", ...)
+
+write.text(x, path, ...)
+
+
+
+Arguments
+
+
+x
+
+A SparkDataFrame
+
+path
+
+The directory where the file is saved
+
+mode
+
+one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by 
default)
+
+...
+
+additional argument(s) passed to the method.
+
+
+
+
+Note
+
+write.text since 2.0.0
+
+
+
+See Also
+
+Other SparkDataFrame functions: $,
+$,SparkDataFrame-method, $-,
+$-,SparkDataFrame-method,
+select, select,
+select,SparkDataFrame,Column-method,
+select,SparkDataFrame,character-method,
+select,SparkDataFrame,list-method;
+SparkDataFrame-class; [,
+[,SparkDataFrame-method, [[,
+[[,SparkDataFrame,numericOrcharacter-method,
+[[-,
+[[-,SparkDataFrame,numericOrcharacter-method,
+subset, subset,
+subset,SparkDataFrame-method;
+agg, agg, agg,
+agg,GroupedData-method,
+agg,SparkDataFrame-method,
+summarize, summarize,
+summarize,
+summarize,GroupedData-method,
+summarize,SparkDataFrame-method;
+arrange, arrange,
+arrange,
+arrange,SparkDataFrame,Column-method,
+arrange,SparkDataFrame,character-method,
+orderBy,SparkDataFrame,characterOrColumn-method;
+as.data.frame,
+as.data.frame,SparkDataFrame-method;
+attach,
+attach,SparkDataFrame-method;
+cache, cache,
+cache,SparkDataFrame-method;
+checkpoint, checkpoint,
+checkpoint,SparkDataFrame-method;
+coalesce, coalesce,
+coalesce,
+coalesce,Column-method,
+coalesce,SparkDataFrame-method;
+collect, collect,
+collect,SparkDataFrame-method;
+colnames, colnames,
+colnames,SparkDataFrame-method,
+colnames-, colnames-,
+colnames-,SparkDataFrame-method,
+columns, columns,
+columns,SparkDataFrame-method,
+names,
+names,SparkDataFrame-method,
+names-,
+names-,SparkDataFrame-method;
+coltypes, coltypes,
+coltypes,SparkDataFrame-method,
+coltypes-, coltypes-,
+coltypes-,SparkDataFrame,character-method;
+count,SparkDataFrame-method,
+nrow, nrow,
+nrow,SparkDataFrame-method;
+createOrReplaceTempView,
+createOrReplaceTempView,
+createOrReplaceTempView,SparkDataFrame,character-method;
+crossJoin,
+crossJoin,SparkDataFrame,SparkDataFrame-method;
+dapplyCollect, dapplyCollect,
+dapplyCollect,SparkDataFrame,function-method;
+dapply, dapply,
+dapply,SparkDataFrame,function,structType-method;
+describe, describe,
+describe,
+describe,SparkDataFrame,ANY-method,
+describe,SparkDataFrame,character-method,
+describe,SparkDataFrame-method,
+summary, summary,
+summary,SparkDataFrame-method;
+dim,
+dim,SparkDataFrame-method;
+distinct, distinct,
+distinct,SparkDataFrame-method,
+unique,
+unique,SparkDataFrame-method;
+dropDuplicates,
+dropDuplicates,
+dropDuplicates,SparkDataFrame-method;
+dropna, dropna,
+dropna,SparkDataFrame-method,
+fillna, fillna,
+fillna,SparkDataFrame-method,
+na.omit, na.omit,
+na.omit,SparkDataFrame-method;
+drop, drop,
+drop, drop,ANY-method,
+drop,SparkDataFrame-method;
+dtypes, dtypes,
+dtypes,SparkDataFrame-method;
+except, except,
+except,SparkDataFrame,SparkDataFrame-method;
+explain, explain,
+explain,
+explain,SparkDataFrame-method,
+explain,StreamingQuery-method;
+filter, filter,
+filter,SparkDataFrame,characterOrColumn-method,
+where, where,
+where,SparkDataFrame,characterOrColumn-method;
+first, first,
+first,
+first,SparkDataFrame-method,
+first,characterOrColumn-method;
+gapplyCollect, gapplyCollect,
+gapplyCollect,
+gapplyCollect,GroupedData-method,
+gapplyCollect,SparkDataFrame-method;
+gapply, gapply,
+gapply,
+gapply,GroupedData-method,
+gapply,SparkDataFrame-method;
+getNumPartitions,
+getNumPartitions,SparkDataFrame-method;
+groupBy, groupBy,
+groupBy,SparkDataFrame-method,
+group_by, group_by,
+group_by,SparkDataFrame-method;
+head,
+head,SparkDataFrame-method;
+hint, hint,
+hint,SparkDataFrame,character-method;
+histogram,
+histogram,SparkDataFrame,characterOrColumn-method;
+insertInto, insertInto,

[48/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/00frame_toc.html
--
diff --git a/site/docs/2.2.0/api/R/00frame_toc.html 
b/site/docs/2.2.0/api/R/00frame_toc.html
new file mode 100644
index 000..1c1aa6e
--- /dev/null
+++ b/site/docs/2.2.0/api/R/00frame_toc.html
@@ -0,0 +1,439 @@
+
+
+
+
+
+R Documentation of SparkR
+
+
+window.onload = function() {
+  var imgs = document.getElementsByTagName('img'), i, img;
+  for (i = 0; i < imgs.length; i++) {
+img = imgs[i];
+// center an image if it is the only element of its parent
+if (img.parentElement.childElementCount === 1)
+  img.parentElement.style.textAlign = 'center';
+  }
+};
+
+
+
+
+
+
+
+* {
+   font-family: "Trebuchet MS", "Lucida Grande", "Lucida Sans Unicode", 
"Lucida Sans", Arial, sans-serif;
+   font-size: 14px;
+}
+body {
+  padding: 0 5px; 
+  margin: 0 auto; 
+  width: 80%;
+  max-width: 60em; /* 960px */
+}
+
+h1, h2, h3, h4, h5, h6 {
+   color: #666;
+}
+h1, h2 {
+   text-align: center;
+}
+h1 {
+   font-size: x-large;
+}
+h2, h3 {
+   font-size: large;
+}
+h4, h6 {
+   font-style: italic;
+}
+h3 {
+   border-left: solid 5px #ddd;
+   padding-left: 5px;
+   font-variant: small-caps;
+}
+
+p img {
+   display: block;
+   margin: auto;
+}
+
+span, code, pre {
+   font-family: Monaco, "Lucida Console", "Courier New", Courier, 
monospace;
+}
+span.acronym {}
+span.env {
+   font-style: italic;
+}
+span.file {}
+span.option {}
+span.pkg {
+   font-weight: bold;
+}
+span.samp{}
+
+dt, p code {
+   background-color: #F7F7F7;
+}
+
+
+
+
+
+
+
+
+SparkR
+
+
+AFTSurvivalRegressionModel-class
+ALSModel-class
+BisectingKMeansModel-class
+FPGrowthModel-class
+GBTClassificationModel-class
+GBTRegressionModel-class
+GaussianMixtureModel-class
+GeneralizedLinearRegressionModel-class
+GroupedData
+IsotonicRegressionModel-class
+KMeansModel-class
+KSTest-class
+LDAModel-class
+LinearSVCModel-class
+LogisticRegressionModel-class
+MultilayerPerceptronClassificationModel-class
+NaiveBayesModel-class
+RandomForestClassificationModel-class
+RandomForestRegressionModel-class
+SparkDataFrame
+StreamingQuery
+WindowSpec
+abs
+acos
+add_months
+alias
+approxCountDistinct
+approxQuantile
+arrange
+array_contains
+as.data.frame
+ascii
+asin
+atan
+atan2
+attach
+avg
+awaitTermination
+base64
+between
+bin
+bitwiseNOT
+bround
+cache
+cacheTable
+cancelJobGroup
+cast
+cbrt
+ceil
+checkpoint
+clearCache
+clearJobGroup
+coalesce
+collect
+coltypes
+column
+columnfunctions
+columns
+concat
+concat_ws
+conv
+corr
+cos
+cosh
+count
+countDistinct
+cov
+covar_pop
+crc32
+createDataFrame
+createExternalTable-deprecated
+createOrReplaceTempView
+createTable
+crossJoin
+crosstab
+cume_dist
+currentDatabase
+dapply
+dapplyCollect
+date_add
+date_format
+date_sub
+datediff
+dayofmonth
+dayofyear
+decode
+dense_rank
+dim
+distinct
+drop
+dropDuplicates
+dropTempTable-deprecated
+dropTempView
+dtypes
+encode
+endsWith
+except
+exp
+explain
+explode
+expm1
+expr
+factorial
+filter
+first
+fitted
+floor
+format_number
+format_string
+freqItems
+from_json
+from_unixtime
+fromutctimestamp
+gapply
+gapplyCollect
+generateAliasesForIntersectedCols
+getNumPartitions
+glm
+greatest
+groupBy
+hash
+hashCode
+head
+hex
+hint
+histogram
+hour
+hypot
+ifelse
+initcap
+insertInto
+install.spark
+instr
+intersect
+is.nan
+isActive
+isLocal
+isStreaming
+join
+kurtosis
+lag
+last
+lastProgress
+last_day
+lead
+least
+length
+levenshtein
+limit
+listColumns
+listDatabases
+listFunctions
+listTables
+lit
+locate
+log
+log10
+log1p
+log2
+lower
+lpad
+ltrim
+match
+max
+md5
+mean
+merge
+min
+minute
+monotonicallyincreasingid
+month
+months_between
+mutate
+nafunctions
+nanvl
+ncol
+negate
+next_day
+nrow
+ntile
+orderBy
+otherwise
+over
+partitionBy
+percent_rank
+persist
+pivot
+pmod
+posexplode
+predict
+print.jobj
+print.structField
+print.structType
+printSchema
+quarter
+queryName
+rand
+randn
+randomSplit
+rangeBetween
+rank
+rbind
+read.df
+read.jdbc
+read.json
+read.ml
+read.orc
+read.parquet
+read.stream
+read.text
+recoverPartitions
+refreshByPath
+refreshTable
+regexp_extract
+regexp_replace
+registerTempTable-deprecated
+rename
+repartition
+reverse
+rint
+round
+row_number
+rowsBetween
+rpad
+rtrim
+sample
+sampleBy
+saveAsTable
+schema
+sd
+second
+select
+selectExpr
+setCheckpointDir
+setCurrentDatabase
+setJobGroup
+setLogLevel
+sha1
+sha2
+shiftLeft
+shiftRight
+shiftRightUnsigned
+show
+showDF
+sign
+sin
+sinh
+size
+skewness
+sort_array
+soundex
+spark.addFile
+spark.als
+spark.bisectingKmeans
+spark.fpGrowth
+spark.gaussianMixture
+spark.gbt
+spark.getSparkFiles
+spark.getSparkFilesRootDirectory
+spark.glm
+spark.isoreg
+spark.kmeans
+spark.kstest
+spark.lapply
+spark.lda
+spark.logit
+spark.mlp
+spark.naiveBayes
+spark.randomForest
+spark.survreg
+spark.svmLinear
+sparkR.callJMethod

[07/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/org/apache/spark/Accumulable.html
--
diff --git a/site/docs/2.2.0/api/java/org/apache/spark/Accumulable.html 
b/site/docs/2.2.0/api/java/org/apache/spark/Accumulable.html
new file mode 100644
index 000..2bd3cfa
--- /dev/null
+++ b/site/docs/2.2.0/api/java/org/apache/spark/Accumulable.html
@@ -0,0 +1,489 @@
+http://www.w3.org/TR/html4/loose.dtd;>
+
+
+
+
+Accumulable (Spark 2.2.0 JavaDoc)
+
+
+
+
+
+
+var methods = 
{"i0":42,"i1":42,"i2":42,"i3":42,"i4":42,"i5":42,"i6":42,"i7":42,"i8":42};
+var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"],32:["t6","Deprecated Methods"]};
+var altColor = "altColor";
+var rowColor = "rowColor";
+var tableTab = "tableTab";
+var activeTableTab = "activeTableTab";
+
+
+JavaScript is disabled on your browser.
+
+
+
+
+
+Skip navigation links
+
+
+
+
+Overview
+Package
+Class
+Tree
+Deprecated
+Index
+Help
+
+
+
+
+PrevClass
+NextClass
+
+
+Frames
+NoFrames
+
+
+AllClasses
+
+
+
+
+
+
+
+Summary:
+Nested|
+Field|
+Constr|
+Method
+
+
+Detail:
+Field|
+Constr|
+Method
+
+
+
+
+
+
+
+
+org.apache.spark
+Class AccumulableR,T
+
+
+
+Object
+
+
+org.apache.spark.AccumulableR,T
+
+
+
+
+
+
+
+All Implemented Interfaces:
+java.io.Serializable
+
+
+Direct Known Subclasses:
+Accumulator
+
+
+Deprecated.
+use AccumulatorV2. Since 
2.0.0.
+
+
+public class AccumulableR,T
+extends Object
+implements java.io.Serializable
+A data type that can be accumulated, i.e. has a commutative 
and associative "add" operation,
+ but where the result type, R, may be different from the element 
type being added, T.
+ 
+ You must define how to add data, and how to merge two of these together.  For 
some data types,
+ such as a counter, these might be the same operation. In that case, you can 
use the simpler
+ Accumulator. They won't always be the same, 
though -- e.g., imagine you are
+ accumulating a set. You will add items to the set, and you will union two 
sets together.
+ 
+ Operations are not thread-safe.
+ 
+ param:  id ID of this accumulator; for internal use only.
+ param:  initialValue initial value of accumulator
+ param:  param helper object defining how to add elements of type 
R and T
+ param:  name human-readable name for use in Spark's web UI
+ param:  countFailedValues whether to accumulate values from failed tasks. 
This is set to true
+  for system and time metrics like serialization time 
or bytes spilled,
+  and false for things with absolute values like 
number of input rows.
+  This should be used for internal metrics only.
+
+See Also:
+Serialized 
Form
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Summary
+
+Constructors
+
+Constructor and Description
+
+
+Accumulable(RinitialValue,
+   AccumulableParamR,Tparam)
+Deprecated.
+
+
+
+
+
+
+
+
+
+
+Method Summary
+
+All MethodsInstance MethodsConcrete MethodsDeprecated Methods
+
+Modifier and Type
+Method and Description
+
+
+void
+add(Tterm)
+Deprecated.
+Add more data to this accumulator / accumulable
+
+
+
+long
+id()
+Deprecated.
+
+
+
+R
+localValue()
+Deprecated.
+Get the current value of this accumulator from within a 
task.
+
+
+
+void
+merge(Rterm)
+Deprecated.
+Merge two accumulable objects together
+
+
+
+scala.OptionString
+name()
+Deprecated.
+
+
+
+void
+setValue(RnewValue)
+Deprecated.
+Set the accumulator's value.
+
+
+
+String
+toString()
+Deprecated.
+
+
+
+R
+value()
+Deprecated.
+Access the accumulator's current value; only allowed on 
driver.
+
+
+
+R
+zero()
+Deprecated.
+
+
+
+
+
+
+
+Methods inherited from classObject
+equals, getClass, hashCode, notify, notifyAll, wait, wait, 
wait
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+Constructor Detail
+
+
+
+
+
+
+
+Accumulable
+publicAccumulable(RinitialValue,
+   AccumulableParamR,Tparam)
+Deprecated.
+
+
+
+
+
+
+
+
+
+Method Detail
+
+
+
+
+
+id
+publiclongid()
+Deprecated.
+
+
+
+
+
+
+
+name
+publicscala.OptionStringname()
+Deprecated.
+
+
+
+
+
+
+
+zero
+publicRzero()
+Deprecated.
+
+
+
+
+
+
+
+
+
+add
+publicvoidadd(Tterm)
+Deprecated.
+Add more data to this accumulator / accumulable
+
+Parameters:
+term - the data to add
+
+
+
+
+
+
+
+
+
+
+merge
+publicvoidmerge(Rterm)
+Deprecated.
+Merge two accumulable objects together
+ 
+ Normally, a user will not want to use this version, but will instead call 
add.
+
+Parameters:
+term - the other R that will get merged with 
this
+

[08/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/java/lib/jquery.js
--
diff --git a/site/docs/2.2.0/api/java/lib/jquery.js 
b/site/docs/2.2.0/api/java/lib/jquery.js
new file mode 100644
index 000..bc3fbc8
--- /dev/null
+++ b/site/docs/2.2.0/api/java/lib/jquery.js
@@ -0,0 +1,2 @@
+/*! jQuery v1.8.2 jquery.com | jquery.org/license */
+(function(a,b){function G(a){var b=F[a]={};return 
p.each(a.split(s),function(a,c){b[c]=!0}),b}function 
J(a,c,d){if(d===b&===1){var 
e="data-"+c.replace(I,"-$1").toLowerCase();d=a.getAttribute(e);if(typeof 
d=="string"){try{d=d==="true"?!0:d==="false"?!1:d==="null"?null:+d+""===d?+d:H.test(d)?p.parseJSON(d):d}catch(f){}p.data(a,c,d)}else
 d=b}return d}function K(a){var b;for(b in 
a){if(b==="data"&(a[b]))continue;if(b!=="toJSON")return!1}return!0}function
 ba(){return!1}function bb(){return!0}function 
bh(a){return!a||!a.parentNode||a.parentNode.nodeType===11}function bi(a,b){do 
a=a[b];while(a&!==1);return a}function 
bj(a,b,c){b=b||0;if(p.isFunction(b))return p.grep(a,function(a,d){var 
e=!!b.call(a,d,a);return e===c});if(b.nodeType)return 
p.grep(a,function(a,d){return a===b===c});if(typeof b=="string"){var 
d=p.grep(a,function(a){return a.nodeType===1});if(be.test(b))return 
p.filter(b,d,!c);b=p.filter(b,d)}return p.grep(a,function(a,d){return p.inArray(
 a,b)>=0===c})}function bk(a){var 
b=bl.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return
 c}function bC(a,b){return 
a.getElementsByTagName(b)[0]||a.appendChild(a.ownerDocument.createElement(b))}function
 bD(a,b){if(b.nodeType!==1||!p.hasData(a))return;var 
c,d,e,f=p._data(a),g=p._data(b,f),h=f.events;if(h){delete 
g.handle,g.events={};for(c in 
h)for(d=0,e=h[c].length;d").appendTo(e.body),c=b.css("display");b.remove();if(c==="none"||c===""){bI=e.body.appendChild(bI||p.extend(e.createElement("iframe"),{frameBorder:0,width:0,height:0}));if(!bJ||!bI.
 
createElement)bJ=(bI.contentWindow||bI.contentDocument).document,bJ.write(""),bJ.close();b=bJ.body.appendChild(bJ.createElement(a)),c=bH(b,"display"),e.body.removeChild(bI)}return
 bS[a]=c,c}function ci(a,b,c,d){var 
e;if(p.isArray(b))p.each(b,function(b,e){c||ce.test(a)?d(a,e):ci(a+"["+(typeof 
e=="object"?b:"")+"]",e,c,d)});else if(!c&(b)==="object")for(e in 
b)ci(a+"["+e+"]",b[e],c,d);else d(a,b)}function cz(a){return 
function(b,c){typeof b!="string"&&(c=b,b="*");var 

[45/56] [partial] spark-website git commit: Add Spark 2.2.0 docs

2017-07-11 Thread marmbrus
http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/cancelJobGroup.html
--
diff --git a/site/docs/2.2.0/api/R/cancelJobGroup.html 
b/site/docs/2.2.0/api/R/cancelJobGroup.html
new file mode 100644
index 000..121fb86
--- /dev/null
+++ b/site/docs/2.2.0/api/R/cancelJobGroup.html
@@ -0,0 +1,56 @@
+
+R: Cancel active jobs for the specified group
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+cancelJobGroup {SparkR}R 
Documentation
+
+Cancel active jobs for the specified group
+
+Description
+
+Cancel active jobs for the specified group
+
+
+
+Usage
+
+
+## Default S3 method:
+cancelJobGroup(groupId)
+
+
+
+Arguments
+
+
+groupId
+
+the ID of job group to be cancelled
+
+
+
+
+Note
+
+cancelJobGroup since 1.5.0
+
+
+
+Examples
+
+## Not run: 
+##D sparkR.session()
+##D cancelJobGroup(myJobGroup)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/cast.html
--
diff --git a/site/docs/2.2.0/api/R/cast.html b/site/docs/2.2.0/api/R/cast.html
new file mode 100644
index 000..ad5b2c2
--- /dev/null
+++ b/site/docs/2.2.0/api/R/cast.html
@@ -0,0 +1,82 @@
+
+R: Casts the column to a different data type.
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+cast 
{SparkR}R Documentation
+
+Casts the column to a different data type.
+
+Description
+
+Casts the column to a different data type.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+cast(x, dataType)
+
+cast(x, dataType)
+
+
+
+Arguments
+
+
+x
+
+a Column.
+
+dataType
+
+a character object describing the target data type.
+See
+https://spark.apache.org/docs/latest/sparkr.html#data-type-mapping-between-r-and-spark;>
+Spark Data Types for available data types.
+
+
+
+
+Note
+
+cast since 1.4.0
+
+
+
+See Also
+
+Other colum_func: alias,
+alias,Column-method; between,
+between,
+between,Column-method;
+endsWith, endsWith,
+endsWith,Column-method;
+otherwise, otherwise,
+otherwise,Column-method;
+over, over,
+over,Column,WindowSpec-method;
+startsWith, startsWith,
+startsWith,Column-method;
+substr, substr,Column-method
+
+
+
+Examples
+
+## Not run: 
+##D   cast(df$age, string)
+## End(Not run)
+
+
+
+[Package SparkR version 2.2.0 Index]
+

http://git-wip-us.apache.org/repos/asf/spark-website/blob/f7ec1155/site/docs/2.2.0/api/R/cbrt.html
--
diff --git a/site/docs/2.2.0/api/R/cbrt.html b/site/docs/2.2.0/api/R/cbrt.html
new file mode 100644
index 000..3fde252
--- /dev/null
+++ b/site/docs/2.2.0/api/R/cbrt.html
@@ -0,0 +1,122 @@
+
+R: cbrt
+
+
+
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/styles/github.min.css;>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/highlight.min.js";>
+https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.3/languages/r.min.js";>
+hljs.initHighlightingOnLoad();
+
+
+cbrt 
{SparkR}R Documentation
+
+cbrt
+
+Description
+
+Computes the cube-root of the given value.
+
+
+
+Usage
+
+
+## S4 method for signature 'Column'
+cbrt(x)
+
+cbrt(x)
+
+
+
+Arguments
+
+
+x
+
+Column to compute on.
+
+
+
+
+Note
+
+cbrt since 1.4.0
+
+
+
+See Also
+
+Other math_funcs: acos,
+acos,Column-method; asin,
+asin,Column-method; atan2,
+atan2,Column-method; atan,
+atan,Column-method; bin,
+bin, bin,Column-method;
+bround, bround,
+bround,Column-method; ceil,
+ceil, ceil,Column-method,
+ceiling,
+ceiling,Column-method; conv,
+conv,
+conv,Column,numeric,numeric-method;
+corr, corr,
+corr, corr,Column-method,
+corr,SparkDataFrame-method;
+cosh, cosh,Column-method;
+cos, cos,Column-method;
+covar_pop, covar_pop,
+covar_pop,characterOrColumn,characterOrColumn-method;
+cov, cov, cov,
+cov,SparkDataFrame-method,
+cov,characterOrColumn-method,
+covar_samp, covar_samp,
+covar_samp,characterOrColumn,characterOrColumn-method;
+expm1, expm1,Column-method;
+exp, exp,Column-method;
+factorial,
+factorial,Column-method;
+floor, floor,Column-method;
+hex, hex,
+hex,Column-method; hypot,
+hypot, hypot,Column-method;
+log10, log10,Column-method;
+log1p, log1p,Column-method;
+log2, log2,Column-method;
+log, log,Column-method;
+pmod, pmod,
+pmod,Column-method; rint,
+rint, rint,Column-method;
+round, round,Column-method;
+shiftLeft, shiftLeft,
+shiftLeft,Column,numeric-method;
+shiftRightUnsigned,

  1   2   3   4   5   6   7   8   9   10   >