Skip to content

Commit

Permalink
Merge branch 'master' into rk/upstream
Browse files Browse the repository at this point in the history
  • Loading branch information
Robert Kruszewski committed May 19, 2017
2 parents ef51e86 + 3f2cd51 commit cb40201
Show file tree
Hide file tree
Showing 11 changed files with 121 additions and 59 deletions.
2 changes: 1 addition & 1 deletion docs/spark-standalone.md
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo
- `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
- `sbin/start-slave.sh` - Starts a slave instance on the machine the script is executed on.
- `sbin/start-all.sh` - Starts both a master and a number of slaves as described above.
- `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
- `sbin/stop-master.sh` - Stops the master that was started via the `sbin/start-master.sh` script.
- `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file.
- `sbin/stop-all.sh` - Stops both the master and the slaves as described above.

Expand Down
4 changes: 4 additions & 0 deletions docs/streaming-kinesis-integration.md
Original file line number Diff line number Diff line change
Expand Up @@ -216,3 +216,7 @@ de-aggregate records during consumption.
- If no Kinesis checkpoint info exists when the input DStream starts, it will start either from the oldest record available (`InitialPositionInStream.TRIM_HORIZON`) or from the latest tip (`InitialPositionInStream.LATEST`). This is configurable.
- `InitialPositionInStream.LATEST` could lead to missed records if data is added to the stream while no input DStreams are running (and no checkpoint info is being stored).
- `InitialPositionInStream.TRIM_HORIZON` may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency.

#### Kinesis retry configuration
- `spark.streaming.kinesis.retry.waitTime` : Wait time between Kinesis retries as a duration string. When reading from Amazon Kinesis, users may hit `ProvisionedThroughputExceededException`'s, when consuming faster than 5 transactions/second or, exceeding the maximum read rate of 2 MB/second. This configuration can be tweaked to increase the sleep between fetches when a fetch fails to reduce these exceptions. Default is "100ms".
- `spark.streaming.kinesis.retry.maxAttempts` : Max number of retries for Kinesis fetches. This config can also be used to tackle the Kinesis `ProvisionedThroughputExceededException`'s in scenarios mentioned above. It can be increased to have more number of retries for Kinesis reads. Default is 3.
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ public static void main(String[] args) {
// $example off$
userRecs.show();
movieRecs.show();

spark.stop();
}
}
3 changes: 2 additions & 1 deletion examples/src/main/python/parquet_inputformat.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
from __future__ import print_function
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
Expand All @@ -16,6 +15,8 @@
# limitations under the License.
#

from __future__ import print_function

import sys

from pyspark.sql import SparkSession
Expand Down
3 changes: 2 additions & 1 deletion examples/src/main/python/pi.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
from __future__ import print_function
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
Expand All @@ -16,6 +15,8 @@
# limitations under the License.
#

from __future__ import print_function

import sys
from random import random
from operator import add
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,10 +50,17 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
fieldTypes: Seq[DataType],
bufferHolder: String): String = {
val fieldEvals = fieldTypes.zipWithIndex.map { case (dt, i) =>
val fieldName = ctx.freshName("fieldName")
val code = s"final ${ctx.javaType(dt)} $fieldName = ${ctx.getValue(input, dt, i.toString)};"
val isNull = s"$input.isNullAt($i)"
ExprCode(code, isNull, fieldName)
val javaType = ctx.javaType(dt)
val isNullVar = ctx.freshName("isNull")
val valueVar = ctx.freshName("value")
val defaultValue = ctx.defaultValue(dt)
val readValue = ctx.getValue(input, dt, i.toString)
val code =
s"""
boolean $isNullVar = $input.isNullAt($i);
$javaType $valueVar = $isNullVar ? $defaultValue : $readValue;
"""
ExprCode(code, isNullVar, valueVar)
}

s"""
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.expressions.codegen

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.BoundReference
import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
import org.apache.spark.sql.types.{DataType, Decimal, StringType, StructType}
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}

class GenerateUnsafeProjectionSuite extends SparkFunSuite {
test("Test unsafe projection string access pattern") {
val dataType = (new StructType).add("a", StringType)
val exprs = BoundReference(0, dataType, nullable = true) :: Nil
val projection = GenerateUnsafeProjection.generate(exprs)
val result = projection.apply(InternalRow(AlwaysNull))
assert(!result.isNullAt(0))
assert(result.getStruct(0, 1).isNullAt(0))
}
}

object AlwaysNull extends InternalRow {
override def numFields: Int = 1
override def setNullAt(i: Int): Unit = {}
override def copy(): InternalRow = this
override def anyNull: Boolean = true
override def isNullAt(ordinal: Int): Boolean = true
override def update(i: Int, value: Any): Unit = notSupported
override def getBoolean(ordinal: Int): Boolean = notSupported
override def getByte(ordinal: Int): Byte = notSupported
override def getShort(ordinal: Int): Short = notSupported
override def getInt(ordinal: Int): Int = notSupported
override def getLong(ordinal: Int): Long = notSupported
override def getFloat(ordinal: Int): Float = notSupported
override def getDouble(ordinal: Int): Double = notSupported
override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = notSupported
override def getUTF8String(ordinal: Int): UTF8String = notSupported
override def getBinary(ordinal: Int): Array[Byte] = notSupported
override def getInterval(ordinal: Int): CalendarInterval = notSupported
override def getStruct(ordinal: Int, numFields: Int): InternalRow = notSupported
override def getArray(ordinal: Int): ArrayData = notSupported
override def getMap(ordinal: Int): MapData = notSupported
override def get(ordinal: Int, dataType: DataType): AnyRef = notSupported
private def notSupported: Nothing = throw new UnsupportedOperationException
}
Original file line number Diff line number Diff line change
Expand Up @@ -198,33 +198,39 @@ public boolean anyNull() {

@Override
public Decimal getDecimal(int ordinal, int precision, int scale) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getDecimal(rowId, precision, scale);
}

@Override
public UTF8String getUTF8String(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getUTF8String(rowId);
}

@Override
public byte[] getBinary(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getBinary(rowId);
}

@Override
public CalendarInterval getInterval(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
final int months = columns[ordinal].getChildColumn(0).getInt(rowId);
final long microseconds = columns[ordinal].getChildColumn(1).getLong(rowId);
return new CalendarInterval(months, microseconds);
}

@Override
public InternalRow getStruct(int ordinal, int numFields) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getStruct(rowId);
}

@Override
public ArrayData getArray(int ordinal) {
if (columns[ordinal].isNullAt(rowId)) return null;
return columns[ordinal].getArray(rowId);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -273,48 +273,41 @@ private[parquet] object ParquetFilters {
schema: StructType,
predicate: sources.Filter,
int96AsTimestamp: Boolean): Option[FilterPredicate] = {
val dataTypeOf = getFieldMap(schema, int96AsTimestamp)
val nameToType = getFieldMap(schema, int96AsTimestamp)

def canMakeFilterOn(name: String): Boolean = nameToType.contains(name)

// NOTE:
//
// For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`,
// which can be casted to `false` implicitly. Please refer to the `eval` method of these
// operators and the `PruneFilters` rule for details.

// Hyukjin:
// I added [[EqualNullSafe]] with [[org.apache.parquet.filter2.predicate.Operators.Eq]].
// So, it performs equality comparison identically when given [[sources.Filter]] is [[EqualTo]].
// The reason why I did this is, that the actual Parquet filter checks null-safe equality
// comparison.
// So I added this and maybe [[EqualTo]] should be changed. It still seems fine though, because
// physical planning does not set `NULL` to [[EqualTo]] but changes it to [[IsNull]] and etc.
// Probably I missed something and obviously this should be changed.

predicate match {
case sources.IsNull(name) if dataTypeOf.contains(name) =>
makeEq.lift(dataTypeOf(name)).map(_(name, null))
case sources.IsNotNull(name) if dataTypeOf.contains(name) =>
makeNotEq.lift(dataTypeOf(name)).map(_(name, null))

case sources.EqualTo(name, value) if dataTypeOf.contains(name) =>
makeEq.lift(dataTypeOf(name)).map(_(name, value))
case sources.Not(sources.EqualTo(name, value)) if dataTypeOf.contains(name) =>
makeNotEq.lift(dataTypeOf(name)).map(_(name, value))

case sources.EqualNullSafe(name, value) if dataTypeOf.contains(name) =>
makeEq.lift(dataTypeOf(name)).map(_(name, value))
case sources.Not(sources.EqualNullSafe(name, value)) if dataTypeOf.contains(name) =>
makeNotEq.lift(dataTypeOf(name)).map(_(name, value))

case sources.LessThan(name, value) if dataTypeOf.contains(name) =>
makeLt.lift(dataTypeOf(name)).map(_(name, value))
case sources.LessThanOrEqual(name, value) if dataTypeOf.contains(name) =>
makeLtEq.lift(dataTypeOf(name)).map(_(name, value))

case sources.GreaterThan(name, value) if dataTypeOf.contains(name) =>
makeGt.lift(dataTypeOf(name)).map(_(name, value))
case sources.GreaterThanOrEqual(name, value) if dataTypeOf.contains(name) =>
makeGtEq.lift(dataTypeOf(name)).map(_(name, value))
case sources.IsNull(name) if canMakeFilterOn(name) =>
makeEq.lift(nameToType(name)).map(_(name, null))
case sources.IsNotNull(name) if canMakeFilterOn(name) =>
makeNotEq.lift(nameToType(name)).map(_(name, null))

case sources.EqualTo(name, value) if canMakeFilterOn(name) =>
makeEq.lift(nameToType(name)).map(_(name, value))
case sources.Not(sources.EqualTo(name, value)) if canMakeFilterOn(name) =>
makeNotEq.lift(nameToType(name)).map(_(name, value))

case sources.EqualNullSafe(name, value) if canMakeFilterOn(name) =>
makeEq.lift(nameToType(name)).map(_(name, value))
case sources.Not(sources.EqualNullSafe(name, value)) if canMakeFilterOn(name) =>
makeNotEq.lift(nameToType(name)).map(_(name, value))

case sources.LessThan(name, value) if canMakeFilterOn(name) =>
makeLt.lift(nameToType(name)).map(_(name, value))
case sources.LessThanOrEqual(name, value) if canMakeFilterOn(name) =>
makeLtEq.lift(nameToType(name)).map(_(name, value))

case sources.GreaterThan(name, value) if canMakeFilterOn(name) =>
makeGt.lift(nameToType(name)).map(_(name, value))
case sources.GreaterThanOrEqual(name, value) if canMakeFilterOn(name) =>
makeGtEq.lift(nameToType(name)).map(_(name, value))

case sources.And(lhs, rhs) =>
// At here, it is not safe to just convert one side if we do not understand the
Expand All @@ -340,8 +333,8 @@ private[parquet] object ParquetFilters {
.map(FilterApi.not)
.map(LogicalInverseRewriter.rewrite)

case sources.In(name, values) if dataTypeOf.contains(name) =>
makeInSet.lift(dataTypeOf(name)).map(_(name, values.toSet))
case sources.In(name, values) if canMakeFilterOn(name) =>
makeInSet.lift(nameToType(name)).map(_(name, values.toSet))

case _ => None
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit
private var schema: StructType = _

// `ValueWriter`s for all fields of the schema
private var rootFieldWriters: Seq[ValueWriter] = _
private var rootFieldWriters: Array[ValueWriter] = _

// Reusable byte array used to write timestamps as Parquet INT96 values
private val timestampBuffer = new Array[Byte](12)
Expand Down Expand Up @@ -96,7 +96,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit
configuration.get(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key).toBoolean
}

this.rootFieldWriters = schema.map(_.dataType).map(makeWriter)
this.rootFieldWriters = schema.map(_.dataType).map(makeWriter).toArray[ValueWriter]

val messageType = new ParquetSchemaConverter(configuration).convert(schema)
val metadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> schemaString).asJava
Expand All @@ -122,7 +122,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit
}

private def writeFields(
row: InternalRow, schema: StructType, fieldWriters: Seq[ValueWriter]): Unit = {
row: InternalRow, schema: StructType, fieldWriters: Array[ValueWriter]): Unit = {
var i = 0
while (i < row.numFields) {
if (!row.isNullAt(i)) {
Expand Down Expand Up @@ -179,7 +179,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit
makeDecimalWriter(precision, scale)

case t: StructType =>
val fieldWriters = t.map(_.dataType).map(makeWriter)
val fieldWriters = t.map(_.dataType).map(makeWriter).toArray[ValueWriter]
(row: SpecializedGetters, ordinal: Int) =>
consumeGroup {
writeFields(row.getStruct(ordinal, t.length), t, fieldWriters)
Expand Down
11 changes: 0 additions & 11 deletions sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1817,17 +1817,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
checkAnswer(df, Row(BigDecimal(0.0)) :: Nil)
}

test("SPARK-19372: Filter can be executed w/o generated code due to JVM code size limit") {
val N = 400
val rows = Seq(Row.fromSeq(Seq.fill(N)("string")))
val schema = StructType(Seq.tabulate(N)(i => StructField(s"_c$i", StringType)))
val df = spark.createDataFrame(spark.sparkContext.makeRDD(rows), schema)

val filter = (0 until N)
.foldLeft(lit(false))((e, index) => e.or(df.col(df.columns(index)) =!= "string"))
df.filter(filter).count
}

test("SPARK-19893: cannot run set operations with map type") {
val df = spark.range(1).select(map(lit("key"), $"id").as("m"))
val e = intercept[AnalysisException](df.intersect(df))
Expand Down

0 comments on commit cb40201

Please sign in to comment.