Commit graph

22137 commits

Author SHA1 Message Date
hyukjinkwon 56e8f48a43 [SPARK-23695][PYTHON] Fix the error message for Kinesis streaming tests
## What changes were proposed in this pull request?

This PR proposes to fix the error message for Kinesis in PySpark when its jar is missing but explicitly enabled.

```bash
ENABLE_KINESIS_TESTS=1 SPARK_TESTING=1 bin/pyspark pyspark.streaming.tests
```

Before:

```
Skipped test_flume_stream (enable by setting environment variable ENABLE_FLUME_TESTS=1Skipped test_kafka_stream (enable by setting environment variable ENABLE_KAFKA_0_8_TESTS=1Traceback (most recent call last):
  File "/usr/local/Cellar/python/2.7.14_3/Frameworks/Python.framework/Versions/2.7/lib/python2.7/runpy.py", line 174, in _run_module_as_main
    "__main__", fname, loader, pkg_name)
  File "/usr/local/Cellar/python/2.7.14_3/Frameworks/Python.framework/Versions/2.7/lib/python2.7/runpy.py", line 72, in _run_code
    exec code in run_globals
  File "/.../spark/python/pyspark/streaming/tests.py", line 1572, in <module>
    % kinesis_asl_assembly_dir) +
NameError: name 'kinesis_asl_assembly_dir' is not defined
```

After:

```
Skipped test_flume_stream (enable by setting environment variable ENABLE_FLUME_TESTS=1Skipped test_kafka_stream (enable by setting environment variable ENABLE_KAFKA_0_8_TESTS=1Traceback (most recent call last):
  File "/usr/local/Cellar/python/2.7.14_3/Frameworks/Python.framework/Versions/2.7/lib/python2.7/runpy.py", line 174, in _run_module_as_main
    "__main__", fname, loader, pkg_name)
  File "/usr/local/Cellar/python/2.7.14_3/Frameworks/Python.framework/Versions/2.7/lib/python2.7/runpy.py", line 72, in _run_code
    exec code in run_globals
  File "/.../spark/python/pyspark/streaming/tests.py", line 1576, in <module>
    "You need to build Spark with 'build/sbt -Pkinesis-asl "
Exception: Failed to find Spark Streaming Kinesis assembly jar in /.../spark/external/kinesis-asl-assembly. You need to build Spark with 'build/sbt -Pkinesis-asl assembly/package streaming-kinesis-asl-assembly/assembly'or 'build/mvn -Pkinesis-asl package' before running this test.
```

## How was this patch tested?

Manually tested.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20834 from HyukjinKwon/minor-variable.
2018-03-15 10:55:33 -07:00
Yuanjian Li 7c3e8995f1 [SPARK-23533][SS] Add support for changing ContinuousDataReader's startOffset
## What changes were proposed in this pull request?

As discussion in #20675, we need add a new interface `ContinuousDataReaderFactory` to support the requirements of setting start offset in Continuous Processing.

## How was this patch tested?

Existing UT.

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #20689 from xuanyuanking/SPARK-23533.
2018-03-15 00:04:28 -07:00
smallory 4f5bad615b [SPARK-23642][DOCS] AccumulatorV2 subclass isZero scaladoc fix
Added/corrected scaladoc for isZero on the DoubleAccumulator, CollectionAccumulator, and LongAccumulator subclasses of AccumulatorV2, particularly noting where there are requirements in addition to having a value of zero in order to return true.

## What changes were proposed in this pull request?

Three scaladoc comments are updated in AccumulatorV2.scala
No changes outside of comment blocks were made.

## How was this patch tested?

Running "sbt unidoc", fixing style errors found, and reviewing the resulting local scaladoc in firefox.

Author: smallory <s.mallory@gmail.com>

Closes #20790 from smallory/patch-1.
2018-03-15 11:58:54 +09:00
“attilapiros” 279b3db897 [SPARK-22915][MLLIB] Streaming tests for spark.ml.feature, from N to Z
# What changes were proposed in this pull request?

Adds structured streaming tests using testTransformer for these suites:

- NGramSuite
- NormalizerSuite
- OneHotEncoderEstimatorSuite
- OneHotEncoderSuite
- PCASuite
- PolynomialExpansionSuite
- QuantileDiscretizerSuite
- RFormulaSuite
- SQLTransformerSuite
- StandardScalerSuite
- StopWordsRemoverSuite
- StringIndexerSuite
- TokenizerSuite
- RegexTokenizerSuite
- VectorAssemblerSuite
- VectorIndexerSuite
- VectorSizeHintSuite
- VectorSlicerSuite
- Word2VecSuite

# How was this patch tested?

They are unit test.

Author: “attilapiros” <piros.attila.zsolt@gmail.com>

Closes #20686 from attilapiros/SPARK-22915.
2018-03-14 18:36:31 -07:00
Kazuaki Ishizaki 1098933b0a [SPARK-23598][SQL] Make methods in BufferedRowIterator public to avoid runtime error for a large query
## What changes were proposed in this pull request?

This PR fixes runtime error regarding a large query when a generated code has split classes. The issue is `append()`, `stopEarly()`, and other methods are not accessible from split classes that are not subclasses of `BufferedRowIterator`.
This PR fixes this issue by making them `public`.

Before applying the PR, we see the following exception by running the attached program with `CodeGenerator.GENERATED_CLASS_SIZE_THRESHOLD=-1`.
```
  test("SPARK-23598") {
    // When set -1 to CodeGenerator.GENERATED_CLASS_SIZE_THRESHOLD, an exception is thrown
    val df_pet_age = Seq((8, "bat"), (15, "mouse"), (5, "horse")).toDF("age", "name")
    df_pet_age.groupBy("name").avg("age").show()
  }
```

Exception:
```
19:40:52.591 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
19:41:32.319 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.IllegalAccessError: tried to access method org.apache.spark.sql.execution.BufferedRowIterator.shouldStop()Z from class org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1$agg_NestedClass1.agg_doAggregateWithKeys$(generated.java:203)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:160)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:616)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
	at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
	at org.apache.spark.scheduler.Task.run(Task.scala:109)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
...
```

Generated code (line 195 calles `stopEarly()`).
```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=1
/* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
/* 008 */   private scala.collection.Iterator[] inputs;
/* 009 */   private boolean agg_initAgg;
/* 010 */   private boolean agg_bufIsNull;
/* 011 */   private double agg_bufValue;
/* 012 */   private boolean agg_bufIsNull1;
/* 013 */   private long agg_bufValue1;
/* 014 */   private agg_FastHashMap agg_fastHashMap;
/* 015 */   private org.apache.spark.unsafe.KVIterator<UnsafeRow, UnsafeRow> agg_fastHashMapIter;
/* 016 */   private org.apache.spark.unsafe.KVIterator agg_mapIter;
/* 017 */   private org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap agg_hashMap;
/* 018 */   private org.apache.spark.sql.execution.UnsafeKVExternalSorter agg_sorter;
/* 019 */   private scala.collection.Iterator inputadapter_input;
/* 020 */   private boolean agg_agg_isNull11;
/* 021 */   private boolean agg_agg_isNull25;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder[] agg_mutableStateArray1 = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder[2];
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] agg_mutableStateArray2 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[2];
/* 024 */   private UnsafeRow[] agg_mutableStateArray = new UnsafeRow[2];
/* 025 */
/* 026 */   public GeneratedIteratorForCodegenStage1(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 031 */     partitionIndex = index;
/* 032 */     this.inputs = inputs;
/* 033 */
/* 034 */     agg_fastHashMap = new agg_FastHashMap(((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).getTaskMemoryManager(), ((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).getEmptyAggregationBuffer());
/* 035 */     agg_hashMap = ((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).createHashMap();
/* 036 */     inputadapter_input = inputs[0];
/* 037 */     agg_mutableStateArray[0] = new UnsafeRow(1);
/* 038 */     agg_mutableStateArray1[0] = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_mutableStateArray[0], 32);
/* 039 */     agg_mutableStateArray2[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_mutableStateArray1[0], 1);
/* 040 */     agg_mutableStateArray[1] = new UnsafeRow(3);
/* 041 */     agg_mutableStateArray1[1] = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_mutableStateArray[1], 32);
/* 042 */     agg_mutableStateArray2[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(agg_mutableStateArray1[1], 3);
/* 043 */
/* 044 */   }
/* 045 */
/* 046 */   public class agg_FastHashMap {
/* 047 */     private org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch batch;
/* 048 */     private int[] buckets;
/* 049 */     private int capacity = 1 << 16;
/* 050 */     private double loadFactor = 0.5;
/* 051 */     private int numBuckets = (int) (capacity / loadFactor);
/* 052 */     private int maxSteps = 2;
/* 053 */     private int numRows = 0;
/* 054 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[1] /* keyName */), org.apache.spark.sql.types.DataTypes.StringType);
/* 055 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add(((java.lang.String) references[2] /* keyName */), org.apache.spark.sql.types.DataTypes.DoubleType)
/* 056 */     .add(((java.lang.String) references[3] /* keyName */), org.apache.spark.sql.types.DataTypes.LongType);
/* 057 */     private Object emptyVBase;
/* 058 */     private long emptyVOff;
/* 059 */     private int emptyVLen;
/* 060 */     private boolean isBatchFull = false;
/* 061 */
/* 062 */     public agg_FastHashMap(
/* 063 */       org.apache.spark.memory.TaskMemoryManager taskMemoryManager,
/* 064 */       InternalRow emptyAggregationBuffer) {
/* 065 */       batch = org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch
/* 066 */       .allocate(keySchema, valueSchema, taskMemoryManager, capacity);
/* 067 */
/* 068 */       final UnsafeProjection valueProjection = UnsafeProjection.create(valueSchema);
/* 069 */       final byte[] emptyBuffer = valueProjection.apply(emptyAggregationBuffer).getBytes();
/* 070 */
/* 071 */       emptyVBase = emptyBuffer;
/* 072 */       emptyVOff = Platform.BYTE_ARRAY_OFFSET;
/* 073 */       emptyVLen = emptyBuffer.length;
/* 074 */
/* 075 */       buckets = new int[numBuckets];
/* 076 */       java.util.Arrays.fill(buckets, -1);
/* 077 */     }
/* 078 */
/* 079 */     public org.apache.spark.sql.catalyst.expressions.UnsafeRow findOrInsert(UTF8String agg_key) {
/* 080 */       long h = hash(agg_key);
/* 081 */       int step = 0;
/* 082 */       int idx = (int) h & (numBuckets - 1);
/* 083 */       while (step < maxSteps) {
/* 084 */         // Return bucket index if it's either an empty slot or already contains the key
/* 085 */         if (buckets[idx] == -1) {
/* 086 */           if (numRows < capacity && !isBatchFull) {
/* 087 */             // creating the unsafe for new entry
/* 088 */             UnsafeRow agg_result = new UnsafeRow(1);
/* 089 */             org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder agg_holder
/* 090 */             = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(agg_result,
/* 091 */               32);
/* 092 */             org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter agg_rowWriter
/* 093 */             = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(
/* 094 */               agg_holder,
/* 095 */               1);
/* 096 */             agg_holder.reset(); //TODO: investigate if reset or zeroout are actually needed
/* 097 */             agg_rowWriter.zeroOutNullBytes();
/* 098 */             agg_rowWriter.write(0, agg_key);
/* 099 */             agg_result.setTotalSize(agg_holder.totalSize());
/* 100 */             Object kbase = agg_result.getBaseObject();
/* 101 */             long koff = agg_result.getBaseOffset();
/* 102 */             int klen = agg_result.getSizeInBytes();
/* 103 */
/* 104 */             UnsafeRow vRow
/* 105 */             = batch.appendRow(kbase, koff, klen, emptyVBase, emptyVOff, emptyVLen);
/* 106 */             if (vRow == null) {
/* 107 */               isBatchFull = true;
/* 108 */             } else {
/* 109 */               buckets[idx] = numRows++;
/* 110 */             }
/* 111 */             return vRow;
/* 112 */           } else {
/* 113 */             // No more space
/* 114 */             return null;
/* 115 */           }
/* 116 */         } else if (equals(idx, agg_key)) {
/* 117 */           return batch.getValueRow(buckets[idx]);
/* 118 */         }
/* 119 */         idx = (idx + 1) & (numBuckets - 1);
/* 120 */         step++;
/* 121 */       }
/* 122 */       // Didn't find it
/* 123 */       return null;
/* 124 */     }
/* 125 */
/* 126 */     private boolean equals(int idx, UTF8String agg_key) {
/* 127 */       UnsafeRow row = batch.getKeyRow(buckets[idx]);
/* 128 */       return (row.getUTF8String(0).equals(agg_key));
/* 129 */     }
/* 130 */
/* 131 */     private long hash(UTF8String agg_key) {
/* 132 */       long agg_hash = 0;
/* 133 */
/* 134 */       int agg_result = 0;
/* 135 */       byte[] agg_bytes = agg_key.getBytes();
/* 136 */       for (int i = 0; i < agg_bytes.length; i++) {
/* 137 */         int agg_hash1 = agg_bytes[i];
/* 138 */         agg_result = (agg_result ^ (0x9e3779b9)) + agg_hash1 + (agg_result << 6) + (agg_result >>> 2);
/* 139 */       }
/* 140 */
/* 141 */       agg_hash = (agg_hash ^ (0x9e3779b9)) + agg_result + (agg_hash << 6) + (agg_hash >>> 2);
/* 142 */
/* 143 */       return agg_hash;
/* 144 */     }
/* 145 */
/* 146 */     public org.apache.spark.unsafe.KVIterator<UnsafeRow, UnsafeRow> rowIterator() {
/* 147 */       return batch.rowIterator();
/* 148 */     }
/* 149 */
/* 150 */     public void close() {
/* 151 */       batch.close();
/* 152 */     }
/* 153 */
/* 154 */   }
/* 155 */
/* 156 */   protected void processNext() throws java.io.IOException {
/* 157 */     if (!agg_initAgg) {
/* 158 */       agg_initAgg = true;
/* 159 */       long wholestagecodegen_beforeAgg = System.nanoTime();
/* 160 */       agg_nestedClassInstance1.agg_doAggregateWithKeys();
/* 161 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[8] /* aggTime */).add((System.nanoTime() - wholestagecodegen_beforeAgg) / 1000000);
/* 162 */     }
/* 163 */
/* 164 */     // output the result
/* 165 */
/* 166 */     while (agg_fastHashMapIter.next()) {
/* 167 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_fastHashMapIter.getKey();
/* 168 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_fastHashMapIter.getValue();
/* 169 */       wholestagecodegen_nestedClassInstance.agg_doAggregateWithKeysOutput(agg_aggKey, agg_aggBuffer);
/* 170 */
/* 171 */       if (shouldStop()) return;
/* 172 */     }
/* 173 */     agg_fastHashMap.close();
/* 174 */
/* 175 */     while (agg_mapIter.next()) {
/* 176 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 177 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_mapIter.getValue();
/* 178 */       wholestagecodegen_nestedClassInstance.agg_doAggregateWithKeysOutput(agg_aggKey, agg_aggBuffer);
/* 179 */
/* 180 */       if (shouldStop()) return;
/* 181 */     }
/* 182 */
/* 183 */     agg_mapIter.close();
/* 184 */     if (agg_sorter == null) {
/* 185 */       agg_hashMap.free();
/* 186 */     }
/* 187 */   }
/* 188 */
/* 189 */   private wholestagecodegen_NestedClass wholestagecodegen_nestedClassInstance = new wholestagecodegen_NestedClass();
/* 190 */   private agg_NestedClass1 agg_nestedClassInstance1 = new agg_NestedClass1();
/* 191 */   private agg_NestedClass agg_nestedClassInstance = new agg_NestedClass();
/* 192 */
/* 193 */   private class agg_NestedClass1 {
/* 194 */     private void agg_doAggregateWithKeys() throws java.io.IOException {
/* 195 */       while (inputadapter_input.hasNext() && !stopEarly()) {
/* 196 */         InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 197 */         int inputadapter_value = inputadapter_row.getInt(0);
/* 198 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 199 */         UTF8String inputadapter_value1 = inputadapter_isNull1 ?
/* 200 */         null : (inputadapter_row.getUTF8String(1));
/* 201 */
/* 202 */         agg_nestedClassInstance.agg_doConsume(inputadapter_row, inputadapter_value, inputadapter_value1, inputadapter_isNull1);
/* 203 */         if (shouldStop()) return;
/* 204 */       }
/* 205 */
/* 206 */       agg_fastHashMapIter = agg_fastHashMap.rowIterator();
/* 207 */       agg_mapIter = ((org.apache.spark.sql.execution.aggregate.HashAggregateExec) references[0] /* plan */).finishAggregate(agg_hashMap, agg_sorter, ((org.apache.spark.sql.execution.metric.SQLMetric) references[4] /* peakMemory */), ((org.apache.spark.sql.execution.metric.SQLMetric) references[5] /* spillSize */), ((org.apache.spark.sql.execution.metric.SQLMetric) references[6] /* avgHashProbe */));
/* 208 */
/* 209 */     }
/* 210 */
/* 211 */   }
/* 212 */
/* 213 */   private class wholestagecodegen_NestedClass {
/* 214 */     private void agg_doAggregateWithKeysOutput(UnsafeRow agg_keyTerm, UnsafeRow agg_bufferTerm)
/* 215 */     throws java.io.IOException {
/* 216 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[7] /* numOutputRows */).add(1);
/* 217 */
/* 218 */       boolean agg_isNull35 = agg_keyTerm.isNullAt(0);
/* 219 */       UTF8String agg_value37 = agg_isNull35 ?
/* 220 */       null : (agg_keyTerm.getUTF8String(0));
/* 221 */       boolean agg_isNull36 = agg_bufferTerm.isNullAt(0);
/* 222 */       double agg_value38 = agg_isNull36 ?
/* 223 */       -1.0 : (agg_bufferTerm.getDouble(0));
/* 224 */       boolean agg_isNull37 = agg_bufferTerm.isNullAt(1);
/* 225 */       long agg_value39 = agg_isNull37 ?
/* 226 */       -1L : (agg_bufferTerm.getLong(1));
/* 227 */
/* 228 */       agg_mutableStateArray1[1].reset();
/* 229 */
/* 230 */       agg_mutableStateArray2[1].zeroOutNullBytes();
/* 231 */
/* 232 */       if (agg_isNull35) {
/* 233 */         agg_mutableStateArray2[1].setNullAt(0);
/* 234 */       } else {
/* 235 */         agg_mutableStateArray2[1].write(0, agg_value37);
/* 236 */       }
/* 237 */
/* 238 */       if (agg_isNull36) {
/* 239 */         agg_mutableStateArray2[1].setNullAt(1);
/* 240 */       } else {
/* 241 */         agg_mutableStateArray2[1].write(1, agg_value38);
/* 242 */       }
/* 243 */
/* 244 */       if (agg_isNull37) {
/* 245 */         agg_mutableStateArray2[1].setNullAt(2);
/* 246 */       } else {
/* 247 */         agg_mutableStateArray2[1].write(2, agg_value39);
/* 248 */       }
/* 249 */       agg_mutableStateArray[1].setTotalSize(agg_mutableStateArray1[1].totalSize());
/* 250 */       append(agg_mutableStateArray[1]);
/* 251 */
/* 252 */     }
/* 253 */
/* 254 */   }
/* 255 */
/* 256 */   private class agg_NestedClass {
/* 257 */     private void agg_doConsume(InternalRow inputadapter_row, int agg_expr_0, UTF8String agg_expr_1, boolean agg_exprIsNull_1) throws java.io.IOException {
/* 258 */       UnsafeRow agg_unsafeRowAggBuffer = null;
/* 259 */       UnsafeRow agg_fastAggBuffer = null;
/* 260 */
/* 261 */       if (true) {
/* 262 */         if (!agg_exprIsNull_1) {
/* 263 */           agg_fastAggBuffer = agg_fastHashMap.findOrInsert(
/* 264 */             agg_expr_1);
/* 265 */         }
/* 266 */       }
/* 267 */       // Cannot find the key in fast hash map, try regular hash map.
/* 268 */       if (agg_fastAggBuffer == null) {
/* 269 */         // generate grouping key
/* 270 */         agg_mutableStateArray1[0].reset();
/* 271 */
/* 272 */         agg_mutableStateArray2[0].zeroOutNullBytes();
/* 273 */
/* 274 */         if (agg_exprIsNull_1) {
/* 275 */           agg_mutableStateArray2[0].setNullAt(0);
/* 276 */         } else {
/* 277 */           agg_mutableStateArray2[0].write(0, agg_expr_1);
/* 278 */         }
/* 279 */         agg_mutableStateArray[0].setTotalSize(agg_mutableStateArray1[0].totalSize());
/* 280 */         int agg_value7 = 42;
/* 281 */
/* 282 */         if (!agg_exprIsNull_1) {
/* 283 */           agg_value7 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(agg_expr_1.getBaseObject(), agg_expr_1.getBaseOffset(), agg_expr_1.numBytes(), agg_value7);
/* 284 */         }
/* 285 */         if (true) {
/* 286 */           // try to get the buffer from hash map
/* 287 */           agg_unsafeRowAggBuffer =
/* 288 */           agg_hashMap.getAggregationBufferFromUnsafeRow(agg_mutableStateArray[0], agg_value7);
/* 289 */         }
/* 290 */         // Can't allocate buffer from the hash map. Spill the map and fallback to sort-based
/* 291 */         // aggregation after processing all input rows.
/* 292 */         if (agg_unsafeRowAggBuffer == null) {
/* 293 */           if (agg_sorter == null) {
/* 294 */             agg_sorter = agg_hashMap.destructAndCreateExternalSorter();
/* 295 */           } else {
/* 296 */             agg_sorter.merge(agg_hashMap.destructAndCreateExternalSorter());
/* 297 */           }
/* 298 */
/* 299 */           // the hash map had be spilled, it should have enough memory now,
/* 300 */           // try to allocate buffer again.
/* 301 */           agg_unsafeRowAggBuffer = agg_hashMap.getAggregationBufferFromUnsafeRow(
/* 302 */             agg_mutableStateArray[0], agg_value7);
/* 303 */           if (agg_unsafeRowAggBuffer == null) {
/* 304 */             // failed to allocate the first page
/* 305 */             throw new OutOfMemoryError("No enough memory for aggregation");
/* 306 */           }
/* 307 */         }
/* 308 */
/* 309 */       }
/* 310 */
/* 311 */       if (agg_fastAggBuffer != null) {
/* 312 */         // common sub-expressions
/* 313 */         boolean agg_isNull21 = false;
/* 314 */         long agg_value23 = -1L;
/* 315 */         if (!false) {
/* 316 */           agg_value23 = (long) agg_expr_0;
/* 317 */         }
/* 318 */         // evaluate aggregate function
/* 319 */         boolean agg_isNull23 = true;
/* 320 */         double agg_value25 = -1.0;
/* 321 */
/* 322 */         boolean agg_isNull24 = agg_fastAggBuffer.isNullAt(0);
/* 323 */         double agg_value26 = agg_isNull24 ?
/* 324 */         -1.0 : (agg_fastAggBuffer.getDouble(0));
/* 325 */         if (!agg_isNull24) {
/* 326 */           agg_agg_isNull25 = true;
/* 327 */           double agg_value27 = -1.0;
/* 328 */           do {
/* 329 */             boolean agg_isNull26 = agg_isNull21;
/* 330 */             double agg_value28 = -1.0;
/* 331 */             if (!agg_isNull21) {
/* 332 */               agg_value28 = (double) agg_value23;
/* 333 */             }
/* 334 */             if (!agg_isNull26) {
/* 335 */               agg_agg_isNull25 = false;
/* 336 */               agg_value27 = agg_value28;
/* 337 */               continue;
/* 338 */             }
/* 339 */
/* 340 */             boolean agg_isNull27 = false;
/* 341 */             double agg_value29 = -1.0;
/* 342 */             if (!false) {
/* 343 */               agg_value29 = (double) 0;
/* 344 */             }
/* 345 */             if (!agg_isNull27) {
/* 346 */               agg_agg_isNull25 = false;
/* 347 */               agg_value27 = agg_value29;
/* 348 */               continue;
/* 349 */             }
/* 350 */
/* 351 */           } while (false);
/* 352 */
/* 353 */           agg_isNull23 = false; // resultCode could change nullability.
/* 354 */           agg_value25 = agg_value26 + agg_value27;
/* 355 */
/* 356 */         }
/* 357 */         boolean agg_isNull29 = false;
/* 358 */         long agg_value31 = -1L;
/* 359 */         if (!false && agg_isNull21) {
/* 360 */           boolean agg_isNull31 = agg_fastAggBuffer.isNullAt(1);
/* 361 */           long agg_value33 = agg_isNull31 ?
/* 362 */           -1L : (agg_fastAggBuffer.getLong(1));
/* 363 */           agg_isNull29 = agg_isNull31;
/* 364 */           agg_value31 = agg_value33;
/* 365 */         } else {
/* 366 */           boolean agg_isNull32 = true;
/* 367 */           long agg_value34 = -1L;
/* 368 */
/* 369 */           boolean agg_isNull33 = agg_fastAggBuffer.isNullAt(1);
/* 370 */           long agg_value35 = agg_isNull33 ?
/* 371 */           -1L : (agg_fastAggBuffer.getLong(1));
/* 372 */           if (!agg_isNull33) {
/* 373 */             agg_isNull32 = false; // resultCode could change nullability.
/* 374 */             agg_value34 = agg_value35 + 1L;
/* 375 */
/* 376 */           }
/* 377 */           agg_isNull29 = agg_isNull32;
/* 378 */           agg_value31 = agg_value34;
/* 379 */         }
/* 380 */         // update fast row
/* 381 */         if (!agg_isNull23) {
/* 382 */           agg_fastAggBuffer.setDouble(0, agg_value25);
/* 383 */         } else {
/* 384 */           agg_fastAggBuffer.setNullAt(0);
/* 385 */         }
/* 386 */
/* 387 */         if (!agg_isNull29) {
/* 388 */           agg_fastAggBuffer.setLong(1, agg_value31);
/* 389 */         } else {
/* 390 */           agg_fastAggBuffer.setNullAt(1);
/* 391 */         }
/* 392 */       } else {
/* 393 */         // common sub-expressions
/* 394 */         boolean agg_isNull7 = false;
/* 395 */         long agg_value9 = -1L;
/* 396 */         if (!false) {
/* 397 */           agg_value9 = (long) agg_expr_0;
/* 398 */         }
/* 399 */         // evaluate aggregate function
/* 400 */         boolean agg_isNull9 = true;
/* 401 */         double agg_value11 = -1.0;
/* 402 */
/* 403 */         boolean agg_isNull10 = agg_unsafeRowAggBuffer.isNullAt(0);
/* 404 */         double agg_value12 = agg_isNull10 ?
/* 405 */         -1.0 : (agg_unsafeRowAggBuffer.getDouble(0));
/* 406 */         if (!agg_isNull10) {
/* 407 */           agg_agg_isNull11 = true;
/* 408 */           double agg_value13 = -1.0;
/* 409 */           do {
/* 410 */             boolean agg_isNull12 = agg_isNull7;
/* 411 */             double agg_value14 = -1.0;
/* 412 */             if (!agg_isNull7) {
/* 413 */               agg_value14 = (double) agg_value9;
/* 414 */             }
/* 415 */             if (!agg_isNull12) {
/* 416 */               agg_agg_isNull11 = false;
/* 417 */               agg_value13 = agg_value14;
/* 418 */               continue;
/* 419 */             }
/* 420 */
/* 421 */             boolean agg_isNull13 = false;
/* 422 */             double agg_value15 = -1.0;
/* 423 */             if (!false) {
/* 424 */               agg_value15 = (double) 0;
/* 425 */             }
/* 426 */             if (!agg_isNull13) {
/* 427 */               agg_agg_isNull11 = false;
/* 428 */               agg_value13 = agg_value15;
/* 429 */               continue;
/* 430 */             }
/* 431 */
/* 432 */           } while (false);
/* 433 */
/* 434 */           agg_isNull9 = false; // resultCode could change nullability.
/* 435 */           agg_value11 = agg_value12 + agg_value13;
/* 436 */
/* 437 */         }
/* 438 */         boolean agg_isNull15 = false;
/* 439 */         long agg_value17 = -1L;
/* 440 */         if (!false && agg_isNull7) {
/* 441 */           boolean agg_isNull17 = agg_unsafeRowAggBuffer.isNullAt(1);
/* 442 */           long agg_value19 = agg_isNull17 ?
/* 443 */           -1L : (agg_unsafeRowAggBuffer.getLong(1));
/* 444 */           agg_isNull15 = agg_isNull17;
/* 445 */           agg_value17 = agg_value19;
/* 446 */         } else {
/* 447 */           boolean agg_isNull18 = true;
/* 448 */           long agg_value20 = -1L;
/* 449 */
/* 450 */           boolean agg_isNull19 = agg_unsafeRowAggBuffer.isNullAt(1);
/* 451 */           long agg_value21 = agg_isNull19 ?
/* 452 */           -1L : (agg_unsafeRowAggBuffer.getLong(1));
/* 453 */           if (!agg_isNull19) {
/* 454 */             agg_isNull18 = false; // resultCode could change nullability.
/* 455 */             agg_value20 = agg_value21 + 1L;
/* 456 */
/* 457 */           }
/* 458 */           agg_isNull15 = agg_isNull18;
/* 459 */           agg_value17 = agg_value20;
/* 460 */         }
/* 461 */         // update unsafe row buffer
/* 462 */         if (!agg_isNull9) {
/* 463 */           agg_unsafeRowAggBuffer.setDouble(0, agg_value11);
/* 464 */         } else {
/* 465 */           agg_unsafeRowAggBuffer.setNullAt(0);
/* 466 */         }
/* 467 */
/* 468 */         if (!agg_isNull15) {
/* 469 */           agg_unsafeRowAggBuffer.setLong(1, agg_value17);
/* 470 */         } else {
/* 471 */           agg_unsafeRowAggBuffer.setNullAt(1);
/* 472 */         }
/* 473 */
/* 474 */       }
/* 475 */
/* 476 */     }
/* 477 */
/* 478 */   }
/* 479 */
/* 480 */ }
```

## How was this patch tested?

Added UT into `WholeStageCodegenSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #20779 from kiszk/SPARK-23598.
2018-03-13 23:04:16 +01:00
zuotingbing 918fb9beee [SPARK-23547][SQL] Cleanup the .pipeout file when the Hive Session closed
## What changes were proposed in this pull request?

![2018-03-07_121010](https://user-images.githubusercontent.com/24823338/37073232-922e10d2-2200-11e8-8172-6e03aa984b39.png)

when the hive session closed, we should also cleanup the .pipeout file.

## How was this patch tested?

Added test cases.

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #20702 from zuotingbing/SPARK-23547.
2018-03-13 11:31:32 -07:00
Xingbo Jiang 9ddd1e2cea [MINOR][SQL][TEST] Create table using dataSourceName in HadoopFsRelationTest
## What changes were proposed in this pull request?

This PR fixes a minor issue in `HadoopFsRelationTest`, that you should create table using `dataSourceName` instead of `parquet`. The issue won't affect the correctness, but it will generate wrong error message in case the test fails.

## How was this patch tested?

Exsiting tests.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #20780 from jiangxb1987/dataSourceName.
2018-03-13 23:31:08 +09:00
Kazuaki Ishizaki 23370554d0 [SPARK-23656][TEST] Perform assertions in XXH64Suite.testKnownByteArrayInputs() on big endian platform, too
## What changes were proposed in this pull request?

This PR enables assertions in `XXH64Suite.testKnownByteArrayInputs()` on big endian platform, too. The current implementation performs them only on little endian platform. This PR increase test coverage of big endian platform.

## How was this patch tested?

Updated `XXH64Suite`
Tested on big endian platform using JIT compiler or interpreter `-Xint`.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #20804 from kiszk/SPARK-23656.
2018-03-13 15:20:09 +01:00
Marco Gaido 567bd31e0a [SPARK-23412][ML] Add cosine distance to BisectingKMeans
## What changes were proposed in this pull request?

The PR adds the option to specify a distance measure in BisectingKMeans. Moreover, it introduces the ability to use the cosine distance measure in it.

## How was this patch tested?

added UTs + existing UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20600 from mgaido91/SPARK-23412.
2018-03-12 14:53:15 -05:00
Jooseong Kim d5b41aea62 [SPARK-23618][K8S][BUILD] Initialize BUILD_ARGS in docker-image-tool.sh
## What changes were proposed in this pull request?

This change initializes BUILD_ARGS to an empty array when $SPARK_HOME/RELEASE exists.

In function build, "local BUILD_ARGS" effectively creates an array of one element where the first and only element is an empty string, so "${BUILD_ARGS[]}" expands to "" and passes an extra argument to docker.

Setting BUILD_ARGS to an empty array makes "${BUILD_ARGS[]}" expand to nothing.

## How was this patch tested?

Manually tested.

$ cat RELEASE
Spark 2.3.0 (git revision a0d7949896) built for Hadoop 2.7.3
Build flags: -Phadoop-2.7 -Phive -Phive-thriftserver -Pkafka-0-8 -Pmesos -Pyarn -Pkubernetes -Pflume -Psparkr -DzincPort=3036
$ ./bin/docker-image-tool.sh -m t testing build
Sending build context to Docker daemon  256.4MB
...

vanzin

Author: Jooseong Kim <jooseong@pinterest.com>

Closes #20791 from jooseong/SPARK-23618.
2018-03-12 11:31:34 -07:00
Xiayun Sun b304e07e06 [SPARK-23462][SQL] improve missing field error message in StructType
## What changes were proposed in this pull request?

The error message ```s"""Field "$name" does not exist."""``` is thrown when looking up an unknown field in StructType. In the error message, we should also contain the information about which columns/fields exist in this struct.

## How was this patch tested?

Added new unit tests.

Note: I created a new `StructTypeSuite.scala` as I couldn't find an existing suite that's suitable to place these tests. I may be missing something so feel free to propose new locations.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Xiayun Sun <xiayunsun@gmail.com>

Closes #20649 from xysun/SPARK-23462.
2018-03-12 22:13:28 +09:00
DylanGuedes b6f837c9d3 [PYTHON] Changes input variable to not conflict with built-in function
Signed-off-by: DylanGuedes <djmgguedesgmail.com>

## What changes were proposed in this pull request?

Changes variable name conflict: [input is a built-in python function](https://stackoverflow.com/questions/20670732/is-input-a-keyword-in-python).

## How was this patch tested?

I runned the example and it works fine.

Author: DylanGuedes <djmgguedes@gmail.com>

Closes #20775 from DylanGuedes/input_variable.
2018-03-10 19:48:29 +09:00
gatorsmile 1a54f48b67 [SPARK-23510][SQL][FOLLOW-UP] Support Hive 2.2 and Hive 2.3 metastore
## What changes were proposed in this pull request?
In the PR https://github.com/apache/spark/pull/20671, I forgot to update the doc about this new support.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20789 from gatorsmile/docUpdate.
2018-03-09 15:54:55 -08:00
Wang Gengliang 10b0657b03 [SPARK-23624][SQL] Revise doc of method pushFilters in Datasource V2
## What changes were proposed in this pull request?

Revise doc of method pushFilters in SupportsPushDownFilters/SupportsPushDownCatalystFilters

In `FileSourceStrategy`, except `partitionKeyFilters`(the references of which is subset of partition keys), all filters needs to be evaluated after scanning. Otherwise, Spark will get wrong result from data sources like Orc/Parquet.

This PR is to improve the doc.

Author: Wang Gengliang <gengliang.wang@databricks.com>

Closes #20769 from gengliangwang/revise_pushdown_doc.
2018-03-09 15:41:19 -08:00
Michał Świtakowski 2ca9bb083c [SPARK-23173][SQL] Avoid creating corrupt parquet files when loading data from JSON
## What changes were proposed in this pull request?

The from_json() function accepts an additional parameter, where the user might specify the schema. The issue is that the specified schema might not be compatible with data. In particular, the JSON data might be missing data for fields declared as non-nullable in the schema. The from_json() function does not verify the data against such errors. When data with missing fields is sent to the parquet encoder, there is no verification either. The end results is a corrupt parquet file.

To avoid corruptions, make sure that all fields in the user-specified schema are set to be nullable.
Since this changes the behavior of a public function, we need to include it in release notes.
The behavior can be reverted by setting `spark.sql.fromJsonForceNullableSchema=false`

## How was this patch tested?

Added two new tests.

Author: Michał Świtakowski <michal.switakowski@databricks.com>

Closes #20694 from mswit-databricks/SPARK-23173.
2018-03-09 14:29:31 -08:00
Marcelo Vanzin 2c3673680e [SPARK-23630][YARN] Allow user's hadoop conf customizations to take effect.
This change restores functionality that was inadvertently removed as part
of the fix for SPARK-22372.

Also modified an existing unit test to make sure the feature works as intended.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20776 from vanzin/SPARK-23630.
2018-03-09 10:36:38 -08:00
Dilip Biswal d90e77bd0e [SPARK-23271][SQL] Parquet output contains only _SUCCESS file after writing an empty dataframe
## What changes were proposed in this pull request?
Below are the two cases.
``` SQL
case 1

scala> List.empty[String].toDF().rdd.partitions.length
res18: Int = 1
```
When we write the above data frame as parquet, we create a parquet file containing
just the schema of the data frame.

Case 2
``` SQL

scala> val anySchema = StructType(StructField("anyName", StringType, nullable = false) :: Nil)
anySchema: org.apache.spark.sql.types.StructType = StructType(StructField(anyName,StringType,false))
scala> spark.read.schema(anySchema).csv("/tmp/empty_folder").rdd.partitions.length
res22: Int = 0
```
For the 2nd case, since number of partitions = 0, we don't call the write task (the task has logic to create the empty metadata only parquet file)

The fix is to create a dummy single partition RDD and set up the write task based on it to ensure
the metadata-only file.

## How was this patch tested?

A new test is added to DataframeReaderWriterSuite.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #20525 from dilipbiswal/spark-23271.
2018-03-08 14:58:40 -08:00
Marco Gaido e7bbca8896 [SPARK-23602][SQL] PrintToStderr prints value also in interpreted mode
## What changes were proposed in this pull request?

`PrintToStderr` was doing what is it supposed to only when code generation is enabled.
The PR adds the same behavior in interpreted mode too.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20773 from mgaido91/SPARK-23602.
2018-03-08 22:02:28 +01:00
Marco Gaido ea480990e7 [SPARK-23628][SQL] calculateParamLength should not return 1 + num of epressions
## What changes were proposed in this pull request?

There was a bug in `calculateParamLength` which caused it to return always 1 + the number of expressions. This could lead to Exceptions especially with expressions of type long.

## How was this patch tested?

added UT + fixed previous UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20772 from mgaido91/SPARK-23628.
2018-03-08 11:09:15 -08:00
lucio 3be4adf648 [SPARK-22751][ML] Improve ML RandomForest shuffle performance
## What changes were proposed in this pull request?

As I mentioned in [SPARK-22751](https://issues.apache.org/jira/browse/SPARK-22751?jql=project%20%3D%20SPARK%20AND%20component%20%3D%20ML%20AND%20text%20~%20randomforest), there is a shuffle performance problem in ML Randomforest when train a RF in high dimensional data.

The reason is that, in _org.apache.spark.tree.impl.RandomForest_, the function _findSplitsBySorting_ will actually flatmap a sparse vector into a dense vector, then in groupByKey there will be a huge shuffle write size.

To avoid this, we can add a filter in flatmap, to filter out zero value. And in function _findSplitsForContinuousFeature_, we can infer the number of zero value by _metadata_.

In addition, if a feature only contains zero value, _continuousSplits_ will not has the key of feature id. So I add a check when using _continuousSplits_.

## How was this patch tested?
Ran model locally using spark-submit.

Author: lucio <576632108@qq.com>

Closes #20472 from lucio-yz/master.
2018-03-08 08:03:24 -06:00
Marco Gaido 92e7ecbbbd [SPARK-23592][SQL] Add interpreted execution to DecodeUsingSerializer
## What changes were proposed in this pull request?

The PR adds interpreted execution to DecodeUsingSerializer.

## How was this patch tested?
added UT

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20760 from mgaido91/SPARK-23592.
2018-03-08 14:18:14 +01:00
Benjamin Peterson 7013eea11c [SPARK-23522][PYTHON] always use sys.exit over builtin exit
The exit() builtin is only for interactive use. applications should use sys.exit().

## What changes were proposed in this pull request?

All usage of the builtin `exit()` function is replaced by `sys.exit()`.

## How was this patch tested?

I ran `python/run-tests`.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Benjamin Peterson <benjamin@python.org>

Closes #20682 from benjaminp/sys-exit.
2018-03-08 20:38:34 +09:00
Li Jin 2cb23a8f51 [SPARK-23011][SQL][PYTHON] Support alternative function form with group aggregate pandas UDF
## What changes were proposed in this pull request?

This PR proposes to support an alternative function from with group aggregate pandas UDF.

The current form:
```
def foo(pdf):
    return ...
```
Takes a single arg that is a pandas DataFrame.

With this PR, an alternative form is supported:
```
def foo(key, pdf):
    return ...
```
The alternative form takes two argument - a tuple that presents the grouping key, and a pandas DataFrame represents the data.

## How was this patch tested?

GroupbyApplyTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #20295 from icexelloss/SPARK-23011-groupby-apply-key.
2018-03-08 20:29:07 +09:00
hyukjinkwon d6632d185e [SPARK-23380][PYTHON] Adds a conf for Arrow fallback in toPandas/createDataFrame with Pandas DataFrame
## What changes were proposed in this pull request?

This PR adds a configuration to control the fallback of Arrow optimization for `toPandas` and `createDataFrame` with Pandas DataFrame.

## How was this patch tested?

Manually tested and unit tests added.

You can test this by:

**`createDataFrame`**

```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame(pdf, "a: map<string, int>")
```

```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame(pdf, "a: map<string, int>")
```

**`toPandas`**

```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```

```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20678 from HyukjinKwon/SPARK-23380-conf.
2018-03-08 20:22:07 +09:00
Bryan Cutler 9bb239c8b1 [SPARK-23159][PYTHON] Update cloudpickle to v0.4.3
## What changes were proposed in this pull request?

The version of cloudpickle in PySpark was close to version 0.4.0 with some additional backported fixes and some minor additions for Spark related things.  This update removes Spark related changes and matches cloudpickle [v0.4.3](https://github.com/cloudpipe/cloudpickle/releases/tag/v0.4.3):

Changes by updating to 0.4.3 include:
* Fix pickling of named tuples https://github.com/cloudpipe/cloudpickle/pull/113
* Built in type constructors for PyPy compatibility [here](d84980ccaa)
* Fix memoryview support https://github.com/cloudpipe/cloudpickle/pull/122
* Improved compatibility with other cloudpickle versions https://github.com/cloudpipe/cloudpickle/pull/128
* Several cleanups https://github.com/cloudpipe/cloudpickle/pull/121 and [here](c91aaf1104)
* [MRG] Regression on pickling classes from the __main__ module https://github.com/cloudpipe/cloudpickle/pull/149
* BUG: Handle instance methods of builtin types https://github.com/cloudpipe/cloudpickle/pull/154
* Fix <span>#</span>129 : do not silence RuntimeError in dump() https://github.com/cloudpipe/cloudpickle/pull/153

## How was this patch tested?

Existing pyspark.tests using python 2.7.14, 3.5.2, 3.6.3

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20373 from BryanCutler/pyspark-update-cloudpickle-42-SPARK-23159.
2018-03-08 20:19:55 +09:00
Maxim Gekk fe22f32041 [SPARK-23620] Splitting thread dump lines by using the br tag
## What changes were proposed in this pull request?

I propose to replace `'\n'` by the `<br>` tag in generated html of thread dump page. The `<br>` tag will split thread lines in more reliable way. For now it could look like on
<img width="1265" alt="the screen shot" src="https://user-images.githubusercontent.com/1580697/37118202-bcd98fc0-2253-11e8-9e61-c2f946869ee0.png">
 if the html is proxied and `'\n'` is replaced by another whitespace. The changes allow to more easily read and copy stack traces.

## How was this patch tested?

I tested it manually by checking the thread dump page and its source.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #20762 from MaxGekk/br-thread-dump.
2018-03-08 10:50:09 +01:00
jx158167 77c91cc746 [SPARK-23524] Big local shuffle blocks should not be checked for corruption.
## What changes were proposed in this pull request?

In current code, all local blocks will be checked for corruption no matter it's big or not.  The reasons are as below:

Size in FetchResult for local block is set to be 0 (https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala#L327)
SPARK-4105 meant to only check the small blocks(size<maxBytesInFlight/3), but for reason 1, below check will be invalid. https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala#L420

We can fix this and avoid the OOM.

## How was this patch tested?

UT added

Author: jx158167 <jx158167@antfin.com>

Closes #20685 from jinxing64/SPARK-23524.
2018-03-07 20:08:32 -08:00
Xingbo Jiang ac76eff6a8 [SPARK-23525][SQL] Support ALTER TABLE CHANGE COLUMN COMMENT for external hive table
## What changes were proposed in this pull request?

The following query doesn't work as expected:
```
CREATE EXTERNAL TABLE ext_table(a STRING, b INT, c STRING) PARTITIONED BY (d STRING)
LOCATION 'sql/core/spark-warehouse/ext_table';
ALTER TABLE ext_table CHANGE a a STRING COMMENT "new comment";
DESC ext_table;
```
The comment of column `a` is not updated, that's because `HiveExternalCatalog.doAlterTable` ignores table schema changes. To fix the issue, we should call `doAlterTableDataSchema` instead of `doAlterTable`.

## How was this patch tested?

Updated `DDLSuite.testChangeColumn`.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #20696 from jiangxb1987/alterColumnComment.
2018-03-07 13:51:44 -08:00
Marcelo Vanzin c99fc9ad9b [SPARK-23550][CORE] Cleanup Utils.
A few different things going on:
- Remove unused methods.
- Move JSON methods to the only class that uses them.
- Move test-only methods to TestUtils.
- Make getMaxResultSize() a config constant.
- Reuse functionality from existing libraries (JRE or JavaUtils) where possible.

The change also includes changes to a few tests to call `Utils.createTempFile` correctly,
so that temp dirs are created under the designated top-level temp dir instead of
potentially polluting git index.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20706 from vanzin/SPARK-23550.
2018-03-07 13:42:06 -08:00
Liang-Chi Hsieh 53561d27c4 [SPARK-23291][SQL][R] R's substr should not reduce starting position by 1 when calling Scala API
## What changes were proposed in this pull request?

Seems R's substr API treats Scala substr API as zero based and so subtracts the given starting position by 1.

Because Scala's substr API also accepts zero-based starting position (treated as the first element), so the current R's substr test results are correct as they all use 1 as starting positions.

## How was this patch tested?

Modified tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #20464 from viirya/SPARK-23291.
2018-03-07 09:37:42 -08:00
Marco Gaido aff7d81cb7 [SPARK-23591][SQL] Add interpreted execution to EncodeUsingSerializer
## What changes were proposed in this pull request?

The PR adds interpreted execution to EncodeUsingSerializer.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20751 from mgaido91/SPARK-23591.
2018-03-07 18:31:59 +01:00
Takeshi Yamamuro 33c2cb22b3 [SPARK-23611][SQL] Add a helper function to check exception for expr evaluation
## What changes were proposed in this pull request?
This pr added a helper function in `ExpressionEvalHelper` to check exceptions in all the path of expression evaluation.

## How was this patch tested?
Modified the existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20748 from maropu/SPARK-23611.
2018-03-07 13:10:51 +01:00
Yuming Wang 04e71c3160 [MINOR][YARN] Add disable yarn.nodemanager.vmem-check-enabled option to memLimitExceededLogMessage
My spark application sometimes will throw `Container killed by YARN for exceeding memory limits`.
Even I increased `spark.yarn.executor.memoryOverhead` to 10G, this error still happen.  The latest config:
<img width="685" alt="memory-config" src="https://user-images.githubusercontent.com/5399861/36975716-f5c548d2-20b5-11e8-95e5-b228d50917b9.png">

And error message:
```
ExecutorLostFailure (executor 121 exited caused by one of the running tasks) Reason: Container killed by YARN for exceeding memory limits. 30.7 GB of 30 GB physical memory used. Consider boosting spark.yarn.executor.memoryOverhead.
```

This is because of [Linux glibc >= 2.10 (RHEL 6) malloc may show excessive virtual memory usage](https://www.ibm.com/developerworks/community/blogs/kevgrig/entry/linux_glibc_2_10_rhel_6_malloc_may_show_excessive_virtual_memory_usage?lang=en). So disable `yarn.nodemanager.vmem-check-enabled` looks like a good option as [MapR mentioned ](https://mapr.com/blog/best-practices-yarn-resource-management).

This PR add disable `yarn.nodemanager.vmem-check-enabled` option to memLimitExceededLogMessage.

More details:
https://issues.apache.org/jira/browse/YARN-4714
https://stackoverflow.com/a/31450291
https://stackoverflow.com/a/42091255

After this PR:
<img width="898" alt="yarn" src="https://user-images.githubusercontent.com/5399861/36975949-c8e7bbbe-20b6-11e8-9513-9f903b868d8d.png">

N/A

Author: Yuming Wang <yumwang@ebay.com>
Author: Yuming Wang <wgyumg@gmail.com>

Closes #20735 from wangyum/YARN-4714.

Change-Id: Ie10836e2c07b6384d228c3f9e89f802823bd9f16
2018-03-07 17:01:29 +08:00
Marco Gaido 4c587eb488 [SPARK-23590][SQL] Add interpreted execution to CreateExternalRow
## What changes were proposed in this pull request?

The PR adds interpreted execution to CreateExternalRow

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20749 from mgaido91/SPARK-23590.
2018-03-06 17:42:17 +01:00
Sean Owen 8bceb899dc [SPARK-23601][BUILD] Remove .md5 files from release
## What changes were proposed in this pull request?

Remove .md5 files from release artifacts

## How was this patch tested?

N/A

Author: Sean Owen <sowen@cloudera.com>

Closes #20737 from srowen/SPARK-23601.
2018-03-06 08:52:28 -06:00
Takeshi Yamamuro e8a259d66d [SPARK-23594][SQL] GetExternalRowField should support interpreted execution
## What changes were proposed in this pull request?
This pr added interpreted execution for `GetExternalRowField`.

## How was this patch tested?
Added tests in `ObjectExpressionsSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20746 from maropu/SPARK-23594.
2018-03-06 13:55:13 +01:00
Wenchen Fan ad640a5aff [SPARK-23303][SQL] improve the explain result for data source v2 relations
## What changes were proposed in this pull request?

The proposed explain format:
**[streaming header] [RelationV2/ScanV2] [data source name] [output] [pushed filters] [options]**

**streaming header**: if it's a streaming relation, put a "Streaming" at the beginning.
**RelationV2/ScanV2**: if it's a logical plan, put a "RelationV2", else, put a "ScanV2"
**data source name**: the simple class name of the data source implementation
**output**: a string of the plan output attributes
**pushed filters**: a string of all the filters that have been pushed to this data source
**options**: all the options to create the data source reader.

The current explain result for data source v2 relation is unreadable:
```
== Parsed Logical Plan ==
'Filter ('i > 6)
+- AnalysisBarrier
      +- Project [j#1]
         +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Filter (i#0 > 6)
   +- Project [j#1, i#0]
      +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Optimized Logical Plan ==
Project [j#1]
+- Filter isnotnull(i#0)
   +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Physical Plan ==
*(1) Project [j#1]
+- *(1) Filter isnotnull(i#0)
   +- *(1) DataSourceV2Scan [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940
```

after this PR
```
== Parsed Logical Plan ==
'Project [unresolvedalias('j, None)]
+- AnalysisBarrier
      +- RelationV2 AdvancedDataSourceV2[i#0, j#1]

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- RelationV2 AdvancedDataSourceV2[i#0, j#1]

== Optimized Logical Plan ==
RelationV2 AdvancedDataSourceV2[j#1]

== Physical Plan ==
*(1) ScanV2 AdvancedDataSourceV2[j#1]
```
-------
```
== Analyzed Logical Plan ==
i: int, j: int
Filter (i#88 > 3)
+- RelationV2 JavaAdvancedDataSourceV2[i#88, j#89]

== Optimized Logical Plan ==
Filter isnotnull(i#88)
+- RelationV2 JavaAdvancedDataSourceV2[i#88, j#89] (Pushed Filters: [GreaterThan(i,3)])

== Physical Plan ==
*(1) Filter isnotnull(i#88)
+- *(1) ScanV2 JavaAdvancedDataSourceV2[i#88, j#89] (Pushed Filters: [GreaterThan(i,3)])
```

an example for streaming query
```
== Parsed Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming RelationV2 MemoryStreamDataSource[value#25]

== Analyzed Logical Plan ==
value: string, count(1): bigint
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming RelationV2 MemoryStreamDataSource[value#25]

== Optimized Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject value#25.toString, obj#4: java.lang.String
         +- Streaming RelationV2 MemoryStreamDataSource[value#25]

== Physical Plan ==
*(4) HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#11L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *(3) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5]
         +- *(2) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *(1) HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#16L])
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *(1) MapElements <function1>, obj#5: java.lang.String
                        +- *(1) DeserializeToObject value#25.toString, obj#4: java.lang.String
                           +- *(1) ScanV2 MemoryStreamDataSource[value#25]
```
## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20647 from cloud-fan/explain.
2018-03-05 20:35:14 -08:00
Henry Robinson 8c5b34c425 [SPARK-23604][SQL] Change Statistics.isEmpty to !Statistics.hasNonNul…
…lValue

## What changes were proposed in this pull request?

Parquet 1.9 will change the semantics of Statistics.isEmpty slightly
to reflect if the null value count has been set. That breaks a
timestamp interoperability test that cares only about whether there
are column values present in the statistics of a written file for an
INT96 column. Fix by using Statistics.hasNonNullValue instead.

## How was this patch tested?

Unit tests continue to pass against Parquet 1.8, and also pass against
a Parquet build including PARQUET-1217.

Author: Henry Robinson <henry@cloudera.com>

Closes #20740 from henryr/spark-23604.
2018-03-05 16:49:24 -08:00
Marco Gaido f6b49f9d1b [SPARK-23586][SQL] Add interpreted execution to WrapOption
## What changes were proposed in this pull request?

The PR adds interpreted execution to WrapOption.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20741 from mgaido91/SPARK-23586_2.
2018-03-06 01:37:51 +01:00
Yogesh Garg 7706eea6a8 [SPARK-18630][PYTHON][ML] Move del method from JavaParams to JavaWrapper; add tests
The `__del__` method that explicitly detaches the object was moved from `JavaParams` to `JavaWrapper` class, this way model summaries could also be garbage collected in Java. A test case was added to make sure that relevant error messages are thrown after the objects are deleted.

I ran pyspark tests  agains `pyspark-ml` module
`./python/run-tests --python-executables=$(which python) --modules=pyspark-ml`

Author: Yogesh Garg <yogesh(dot)garg()databricks(dot)com>

Closes #20724 from yogeshg/java_wrapper_memory.
2018-03-05 15:53:10 -08:00
Marcelo Vanzin 508573958d [SPARK-23538][CORE] Remove custom configuration for SSL client.
These options were used to configure the built-in JRE SSL libraries
when downloading files from HTTPS servers. But because they were also
used to set up the now (long) removed internal HTTPS file server,
their default configuration chose convenience over security by having
overly lenient settings.

This change removes the configuration options that affect the JRE SSL
libraries. The JRE trust store can still be configured via system
properties (or globally in the JRE security config). The only lost
functionality is not being able to disable the default hostname
verifier when using spark-submit, which should be fine since Spark
itself is not using https for any internal functionality anymore.

I also removed the HTTP-related code from the REPL class loader, since
we haven't had a HTTP server for REPL-generated classes for a while.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20723 from vanzin/SPARK-23538.
2018-03-05 15:03:27 -08:00
Xianjin YE f2cab56ca2 [SPARK-23040][CORE] Returns interruptible iterator for shuffle reader
## What changes were proposed in this pull request?

Before this commit, a non-interruptible iterator is returned if aggregator or ordering is specified.
This commit also ensures that sorter is closed even when task is cancelled(killed) in the middle of sorting.

## How was this patch tested?

Add a unit test in JobCancellationSuite

Author: Xianjin YE <advancedxy@gmail.com>

Closes #20449 from advancedxy/SPARK-23040.
2018-03-05 14:57:32 -08:00
Jose Torres b0f422c386 [SPARK-23559][SS] Add epoch ID to DataWriterFactory.
## What changes were proposed in this pull request?

Add an epoch ID argument to DataWriterFactory for use in streaming. As a side effect of passing in this value, DataWriter will now have a consistent lifecycle; commit() or abort() ends the lifecycle of a DataWriter instance in any execution mode.

I considered making a separate streaming interface and adding the epoch ID only to that one, but I think it requires a lot of extra work for no real gain. I think it makes sense to define epoch 0 as the one and only epoch of a non-streaming query.

## How was this patch tested?

existing unit tests

Author: Jose Torres <jose@databricks.com>

Closes #20710 from jose-torres/api2.
2018-03-05 13:23:01 -08:00
Marco Gaido ba622f45ca [SPARK-23585][SQL] Add interpreted execution to UnwrapOption
## What changes were proposed in this pull request?

The PR adds interpreted execution to UnwrapOption.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20736 from mgaido91/SPARK-23586.
2018-03-05 20:43:03 +01:00
WeichenXu 98a5c0a35f [SPARK-22882][ML][TESTS] ML test for structured streaming: ml.classification
## What changes were proposed in this pull request?

adding Structured Streaming tests for all Models/Transformers in spark.ml.classification

## How was this patch tested?

N/A

Author: WeichenXu <weichen.xu@databricks.com>

Closes #20121 from WeichenXu123/ml_stream_test_classification.
2018-03-05 10:50:00 -08:00
Rekha Joshi 4586eada42 [SPARK-22430][R][DOCS] Unknown tag warnings when building R docs with Roxygen 6.0.1
## What changes were proposed in this pull request?
Removed export tag to get rid of unknown tag warnings

## How was this patch tested?
Existing tests

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: rjoshi2 <rekhajoshm@gmail.com>

Closes #20501 from rekhajoshm/SPARK-22430.
2018-03-05 09:30:49 -08:00
Yuming Wang 947b4e6f09 [SPARK-23510][DOC][FOLLOW-UP] Update spark.sql.hive.metastore.version
## What changes were proposed in this pull request?
Update `spark.sql.hive.metastore.version` to 2.3.2, same as HiveUtils.scala:
ff1480189b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala (L63-L65)

## How was this patch tested?

N/A

Author: Yuming Wang <yumwang@ebay.com>

Closes #20734 from wangyum/SPARK-23510-FOLLOW-UP.
2018-03-05 16:21:02 +01:00
Mihaly Toth a366b950b9 [SPARK-23329][SQL] Fix documentation of trigonometric functions
## What changes were proposed in this pull request?

Provide more details in trigonometric function documentations. Referenced `java.lang.Math` for further details in the descriptions.
## How was this patch tested?

Ran full build, checked generated documentation manually

Author: Mihaly Toth <misutoth@gmail.com>

Closes #20618 from misutoth/trigonometric-doc.
2018-03-05 23:46:40 +09:00
Anirudh 5ff72ffcf4 [SPARK-23566][MINOR][DOC] Argument name mismatch fixed
Argument name mismatch fixed.

## What changes were proposed in this pull request?

`col` changed to `new` in doc string to match the argument list.

Patch file added: https://issues.apache.org/jira/browse/SPARK-23566

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Anirudh <animenon@mail.com>

Closes #20716 from animenon/master.
2018-03-05 23:17:16 +09:00
Ala Luszczak 42cf48e20c [SPARK-23496][CORE] Locality of coalesced partitions can be severely skewed by the order of input partitions
## What changes were proposed in this pull request?

The algorithm in `DefaultPartitionCoalescer.setupGroups` is responsible for picking preferred locations for coalesced partitions. It analyzes the preferred locations of input partitions. It starts by trying to create one partition for each unique location in the input. However, if the the requested number of coalesced partitions is higher that the number of unique locations, it has to pick duplicate locations.

Previously, the duplicate locations would be picked by iterating over the input partitions in order, and copying their preferred locations to coalesced partitions. If the input partitions were clustered by location, this could result in severe skew.

With the fix, instead of iterating over the list of input partitions in order, we pick them at random. It's not perfectly balanced, but it's much better.

## How was this patch tested?

Unit test reproducing the behavior was added.

Author: Ala Luszczak <ala@databricks.com>

Closes #20664 from ala/SPARK-23496.
2018-03-05 14:33:12 +01:00