aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-20253][SQL] Remove unnecessary nullchecks of a return value from ↵Kazuaki Ishizaki2017-04-104-33/+51
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Spark runtime routines in generated Java code ## What changes were proposed in this pull request? This PR elminates unnecessary nullchecks of a return value from known Spark runtime routines. We know whether a given Spark runtime routine returns ``null`` or not (e.g. ``ArrayData.toDoubleArray()`` never returns ``null``). Thus, we can eliminate a null check for the return value from the Spark runtime routine. When we run the following example program, now we get the Java code "Without this PR". In this code, since we know ``ArrayData.toDoubleArray()`` never returns ``null```, we can eliminate null checks at lines 90-92, and 97. ```java val ds = sparkContext.parallelize(Seq(Array(1.1, 2.2)), 1).toDS.cache ds.count ds.map(e => e).show ``` Without this PR ```java /* 050 */ protected void processNext() throws java.io.IOException { /* 051 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 052 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 053 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 054 */ ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0)); /* 055 */ /* 056 */ ArrayData deserializetoobject_value1 = null; /* 057 */ /* 058 */ if (!inputadapter_isNull) { /* 059 */ int deserializetoobject_dataLength = inputadapter_value.numElements(); /* 060 */ /* 061 */ Double[] deserializetoobject_convertedArray = null; /* 062 */ deserializetoobject_convertedArray = new Double[deserializetoobject_dataLength]; /* 063 */ /* 064 */ int deserializetoobject_loopIndex = 0; /* 065 */ while (deserializetoobject_loopIndex < deserializetoobject_dataLength) { /* 066 */ MapObjects_loopValue2 = (double) (inputadapter_value.getDouble(deserializetoobject_loopIndex)); /* 067 */ MapObjects_loopIsNull2 = inputadapter_value.isNullAt(deserializetoobject_loopIndex); /* 068 */ /* 069 */ if (MapObjects_loopIsNull2) { /* 070 */ throw new RuntimeException(((java.lang.String) references[0])); /* 071 */ } /* 072 */ if (false) { /* 073 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null; /* 074 */ } else { /* 075 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue2; /* 076 */ } /* 077 */ /* 078 */ deserializetoobject_loopIndex += 1; /* 079 */ } /* 080 */ /* 081 */ deserializetoobject_value1 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /*###*/ /* 082 */ } /* 083 */ boolean deserializetoobject_isNull = true; /* 084 */ double[] deserializetoobject_value = null; /* 085 */ if (!inputadapter_isNull) { /* 086 */ deserializetoobject_isNull = false; /* 087 */ if (!deserializetoobject_isNull) { /* 088 */ Object deserializetoobject_funcResult = null; /* 089 */ deserializetoobject_funcResult = deserializetoobject_value1.toDoubleArray(); /* 090 */ if (deserializetoobject_funcResult == null) { /* 091 */ deserializetoobject_isNull = true; /* 092 */ } else { /* 093 */ deserializetoobject_value = (double[]) deserializetoobject_funcResult; /* 094 */ } /* 095 */ /* 096 */ } /* 097 */ deserializetoobject_isNull = deserializetoobject_value == null; /* 098 */ } /* 099 */ /* 100 */ boolean mapelements_isNull = true; /* 101 */ double[] mapelements_value = null; /* 102 */ if (!false) { /* 103 */ mapelements_resultIsNull = false; /* 104 */ /* 105 */ if (!mapelements_resultIsNull) { /* 106 */ mapelements_resultIsNull = deserializetoobject_isNull; /* 107 */ mapelements_argValue = deserializetoobject_value; /* 108 */ } /* 109 */ /* 110 */ mapelements_isNull = mapelements_resultIsNull; /* 111 */ if (!mapelements_isNull) { /* 112 */ Object mapelements_funcResult = null; /* 113 */ mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue); /* 114 */ if (mapelements_funcResult == null) { /* 115 */ mapelements_isNull = true; /* 116 */ } else { /* 117 */ mapelements_value = (double[]) mapelements_funcResult; /* 118 */ } /* 119 */ /* 120 */ } /* 121 */ mapelements_isNull = mapelements_value == null; /* 122 */ } /* 123 */ /* 124 */ serializefromobject_resultIsNull = false; /* 125 */ /* 126 */ if (!serializefromobject_resultIsNull) { /* 127 */ serializefromobject_resultIsNull = mapelements_isNull; /* 128 */ serializefromobject_argValue = mapelements_value; /* 129 */ } /* 130 */ /* 131 */ boolean serializefromobject_isNull = serializefromobject_resultIsNull; /* 132 */ final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue); /* 133 */ serializefromobject_isNull = serializefromobject_value == null; /* 134 */ serializefromobject_holder.reset(); /* 135 */ /* 136 */ serializefromobject_rowWriter.zeroOutNullBytes(); /* 137 */ /* 138 */ if (serializefromobject_isNull) { /* 139 */ serializefromobject_rowWriter.setNullAt(0); /* 140 */ } else { /* 141 */ // Remember the current cursor so that we can calculate how many bytes are /* 142 */ // written later. /* 143 */ final int serializefromobject_tmpCursor = serializefromobject_holder.cursor; /* 144 */ /* 145 */ if (serializefromobject_value instanceof UnsafeArrayData) { /* 146 */ final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes(); /* 147 */ // grow the global buffer before writing data. /* 148 */ serializefromobject_holder.grow(serializefromobject_sizeInBytes); /* 149 */ ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor); /* 150 */ serializefromobject_holder.cursor += serializefromobject_sizeInBytes; /* 151 */ /* 152 */ } else { /* 153 */ final int serializefromobject_numElements = serializefromobject_value.numElements(); /* 154 */ serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8); /* 155 */ /* 156 */ for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) { /* 157 */ if (serializefromobject_value.isNullAt(serializefromobject_index)) { /* 158 */ serializefromobject_arrayWriter.setNullDouble(serializefromobject_index); /* 159 */ } else { /* 160 */ final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index); /* 161 */ serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element); /* 162 */ } /* 163 */ } /* 164 */ } /* 165 */ /* 166 */ serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor); /* 167 */ } /* 168 */ serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize()); /* 169 */ append(serializefromobject_result); /* 170 */ if (shouldStop()) return; /* 171 */ } /* 172 */ } ``` With this PR (removed most of lines 90-97 in the above code) ```java /* 050 */ protected void processNext() throws java.io.IOException { /* 051 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 052 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 053 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 054 */ ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0)); /* 055 */ /* 056 */ ArrayData deserializetoobject_value1 = null; /* 057 */ /* 058 */ if (!inputadapter_isNull) { /* 059 */ int deserializetoobject_dataLength = inputadapter_value.numElements(); /* 060 */ /* 061 */ Double[] deserializetoobject_convertedArray = null; /* 062 */ deserializetoobject_convertedArray = new Double[deserializetoobject_dataLength]; /* 063 */ /* 064 */ int deserializetoobject_loopIndex = 0; /* 065 */ while (deserializetoobject_loopIndex < deserializetoobject_dataLength) { /* 066 */ MapObjects_loopValue2 = (double) (inputadapter_value.getDouble(deserializetoobject_loopIndex)); /* 067 */ MapObjects_loopIsNull2 = inputadapter_value.isNullAt(deserializetoobject_loopIndex); /* 068 */ /* 069 */ if (MapObjects_loopIsNull2) { /* 070 */ throw new RuntimeException(((java.lang.String) references[0])); /* 071 */ } /* 072 */ if (false) { /* 073 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null; /* 074 */ } else { /* 075 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue2; /* 076 */ } /* 077 */ /* 078 */ deserializetoobject_loopIndex += 1; /* 079 */ } /* 080 */ /* 081 */ deserializetoobject_value1 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /*###*/ /* 082 */ } /* 083 */ boolean deserializetoobject_isNull = true; /* 084 */ double[] deserializetoobject_value = null; /* 085 */ if (!inputadapter_isNull) { /* 086 */ deserializetoobject_isNull = false; /* 087 */ if (!deserializetoobject_isNull) { /* 088 */ Object deserializetoobject_funcResult = null; /* 089 */ deserializetoobject_funcResult = deserializetoobject_value1.toDoubleArray(); /* 090 */ deserializetoobject_value = (double[]) deserializetoobject_funcResult; /* 091 */ /* 092 */ } /* 093 */ /* 094 */ } /* 095 */ /* 096 */ boolean mapelements_isNull = true; /* 097 */ double[] mapelements_value = null; /* 098 */ if (!false) { /* 099 */ mapelements_resultIsNull = false; /* 100 */ /* 101 */ if (!mapelements_resultIsNull) { /* 102 */ mapelements_resultIsNull = deserializetoobject_isNull; /* 103 */ mapelements_argValue = deserializetoobject_value; /* 104 */ } /* 105 */ /* 106 */ mapelements_isNull = mapelements_resultIsNull; /* 107 */ if (!mapelements_isNull) { /* 108 */ Object mapelements_funcResult = null; /* 109 */ mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue); /* 110 */ if (mapelements_funcResult == null) { /* 111 */ mapelements_isNull = true; /* 112 */ } else { /* 113 */ mapelements_value = (double[]) mapelements_funcResult; /* 114 */ } /* 115 */ /* 116 */ } /* 117 */ mapelements_isNull = mapelements_value == null; /* 118 */ } /* 119 */ /* 120 */ serializefromobject_resultIsNull = false; /* 121 */ /* 122 */ if (!serializefromobject_resultIsNull) { /* 123 */ serializefromobject_resultIsNull = mapelements_isNull; /* 124 */ serializefromobject_argValue = mapelements_value; /* 125 */ } /* 126 */ /* 127 */ boolean serializefromobject_isNull = serializefromobject_resultIsNull; /* 128 */ final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue); /* 129 */ serializefromobject_isNull = serializefromobject_value == null; /* 130 */ serializefromobject_holder.reset(); /* 131 */ /* 132 */ serializefromobject_rowWriter.zeroOutNullBytes(); /* 133 */ /* 134 */ if (serializefromobject_isNull) { /* 135 */ serializefromobject_rowWriter.setNullAt(0); /* 136 */ } else { /* 137 */ // Remember the current cursor so that we can calculate how many bytes are /* 138 */ // written later. /* 139 */ final int serializefromobject_tmpCursor = serializefromobject_holder.cursor; /* 140 */ /* 141 */ if (serializefromobject_value instanceof UnsafeArrayData) { /* 142 */ final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes(); /* 143 */ // grow the global buffer before writing data. /* 144 */ serializefromobject_holder.grow(serializefromobject_sizeInBytes); /* 145 */ ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor); /* 146 */ serializefromobject_holder.cursor += serializefromobject_sizeInBytes; /* 147 */ /* 148 */ } else { /* 149 */ final int serializefromobject_numElements = serializefromobject_value.numElements(); /* 150 */ serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8); /* 151 */ /* 152 */ for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) { /* 153 */ if (serializefromobject_value.isNullAt(serializefromobject_index)) { /* 154 */ serializefromobject_arrayWriter.setNullDouble(serializefromobject_index); /* 155 */ } else { /* 156 */ final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index); /* 157 */ serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element); /* 158 */ } /* 159 */ } /* 160 */ } /* 161 */ /* 162 */ serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor); /* 163 */ } /* 164 */ serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize()); /* 165 */ append(serializefromobject_result); /* 166 */ if (shouldStop()) return; /* 167 */ } /* 168 */ } ``` ## How was this patch tested? Add test suites to ``DatasetPrimitiveSuite`` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #17569 from kiszk/SPARK-20253.
* [SPARK-20260][MLLIB] String interpolation required for error messageVijay Ramesh2017-04-097-10/+10
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This error message doesn't get properly formatted because of a missing `s`. Currently the error looks like: ``` Caused by: java.lang.IllegalArgumentException: requirement failed: indices should be one-based and in ascending order; found current=$current, previous=$previous; line="$line" ``` (note the literal `$current` instead of the interpolated value) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Vijay Ramesh <vramesh@demandbase.com> Closes #17572 from vijaykramesh/master.
* [SPARK-19991][CORE][YARN] FileSegmentManagedBuffer performance improvementSean Owen2017-04-093-1/+13
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid `NoSuchElementException` every time `ConfigProvider.get(val, default)` falls back to default. This apparently causes non-trivial overhead in at least one path, and can easily be avoided. See https://github.com/apache/spark/pull/17329 ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #17567 from srowen/SPARK-19991.
* [MINOR] Issue: Change "slice" vs "partition" in exception messages (and code?)asmith262017-04-097-7/+7
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Came across the term "slice" when running some spark scala code. Consequently, a Google search indicated that "slices" and "partitions" refer to the same things; indeed see: - [This issue](https://issues.apache.org/jira/browse/SPARK-1701) - [This pull request](https://github.com/apache/spark/pull/2305) - [This StackOverflow answer](http://stackoverflow.com/questions/23436640/what-is-the-difference-between-an-rdd-partition-and-a-slice) and [this one](http://stackoverflow.com/questions/24269495/what-are-the-differences-between-slices-and-partitions-of-rdds) Thus this pull request fixes the occurrence of slice I came accross. Nonetheless, [it would appear](https://github.com/apache/spark/search?utf8=%E2%9C%93&q=slice&type=) there are still many references to "slice/slices" - thus I thought I'd raise this Pull Request to address the issue (sorry if this is the wrong place, I'm not too familar with raising apache issues). ## How was this patch tested? (Not tested locally - only a minor exception message change.) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: asmith26 <asmith26@users.noreply.github.com> Closes #17565 from asmith26/master.
* [SPARK-20262][SQL] AssertNotNull should throw NullPointerExceptionReynold Xin2017-04-071-3/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? AssertNotNull currently throws RuntimeException. It should throw NullPointerException, which is more specific. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #17573 from rxin/SPARK-20262.
* [SPARK-20246][SQL] should not push predicate down through aggregate with ↵Wenchen Fan2017-04-072-33/+68
| | | | | | | | | | | | | | | | non-deterministic expressions ## What changes were proposed in this pull request? Similar to `Project`, when `Aggregate` has non-deterministic expressions, we should not push predicate down through it, as it will change the number of input rows and thus change the evaluation result of non-deterministic expressions in `Aggregate`. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17562 from cloud-fan/filter.
* [SPARK-20255] Move listLeafFiles() to InMemoryFileIndexAdrian Ionescu2017-04-073-231/+236
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request Trying to get a grip on the `FileIndex` hierarchy, I was confused by the following inconsistency: On the one hand, `PartitioningAwareFileIndex` defines `leafFiles` and `leafDirToChildrenFiles` as abstract, but on the other it fully implements `listLeafFiles` which does all the listing of files. However, the latter is only used by `InMemoryFileIndex`. I'm hereby proposing to move this method (and all its dependencies) to the implementation class that actually uses it, and thus unclutter the `PartitioningAwareFileIndex` interface. ## How was this patch tested? `./build/sbt sql/test` Author: Adrian Ionescu <adrian@databricks.com> Closes #17570 from adrian-ionescu/list-leaf-files.
* [SPARK-20258][DOC][SPARKR] Fix SparkR logistic regression example in ↵actuaryzhang2017-04-071-3/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | programming guide (did not converge) ## What changes were proposed in this pull request? SparkR logistic regression example did not converge in programming guide (for IRWLS). All estimates are essentially zero: ``` training2 <- read.df("data/mllib/sample_binary_classification_data.txt", source = "libsvm") df_list2 <- randomSplit(training2, c(7,3), 2) binomialDF <- df_list2[[1]] binomialTestDF <- df_list2[[2]] binomialGLM <- spark.glm(binomialDF, label ~ features, family = "binomial") 17/04/07 11:42:03 WARN WeightedLeastSquares: Cholesky solver failed due to singular covariance matrix. Retrying with Quasi-Newton solver. > summary(binomialGLM) Coefficients: Estimate (Intercept) 9.0255e+00 features_0 0.0000e+00 features_1 0.0000e+00 features_2 0.0000e+00 features_3 0.0000e+00 features_4 0.0000e+00 features_5 0.0000e+00 features_6 0.0000e+00 features_7 0.0000e+00 ``` Author: actuaryzhang <actuaryzhang10@gmail.com> Closes #17571 from actuaryzhang/programGuide2.
* [SPARK-20197][SPARKR] CRAN check fail with package installationFelix Cheung2017-04-071-2/+3
| | | | | | | | | | ## What changes were proposed in this pull request? Test failed because SPARK_HOME is not set before Spark is installed. Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17516 from felixcheung/rdircheckincran.
* [SPARK-20026][DOC][SPARKR] Add Tweedie example for SparkR in programming guideactuaryzhang2017-04-071-0/+9
| | | | | | | | | | ## What changes were proposed in this pull request? Add Tweedie example for SparkR in programming guide. The doc was already updated in #17103. Author: actuaryzhang <actuaryzhang10@gmail.com> Closes #17553 from actuaryzhang/programGuide.
* [SPARK-20218][DOC][APP-ID] applications//stages' in REST API,add description.郭小龙 102076332017-04-071-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. '/applications/[app-id]/stages' in rest api.status should add description '?status=[active|complete|pending|failed] list only stages in the state.' Now the lack of this description, resulting in the use of this api do not know the use of the status through the brush stage list. 2.'/applications/[app-id]/stages/[stage-id]' in REST API,remove redundant description ‘?status=[active|complete|pending|failed] list only stages in the state.’. Because only one stage is determined based on stage-id. code: GET def stageList(QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = { val listener = ui.jobProgressListener val stageAndStatus = AllStagesResource.stagesAndStatus(ui) val adjStatuses = { if (statuses.isEmpty()) { Arrays.asList(StageStatus.values(): _*) } else { statuses } }; ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn> Closes #17534 from guoxiaolongzte/SPARK-20218.
* [SPARK-20076][ML][PYSPARK] Add Python interface for ml.stats.CorrelationLiang-Chi Hsieh2017-04-072-4/+65
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The Dataframes-based support for the correlation statistics is added in #17108. This patch adds the Python interface for it. ## How was this patch tested? Python unit test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #17494 from viirya/correlation-python-api.
* [SPARK-20245][SQL][MINOR] pass output to LogicalRelation directlyWenchen Fan2017-04-078-55/+49
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently `LogicalRelation` has a `expectedOutputAttributes` parameter, which makes it hard to reason about what the actual output is. Like other leaf nodes, `LogicalRelation` should also take `output` as a parameter, to simplify the logic ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #17552 from cloud-fan/minor.
* [SPARK-19495][SQL] Make SQLConf slightly more extensible - addendumReynold Xin2017-04-061-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? This is a tiny addendum to SPARK-19495 to remove the private visibility for copy, which is the only package private method in the entire file. ## How was this patch tested? N/A - no semantic change. Author: Reynold Xin <rxin@databricks.com> Closes #17555 from rxin/SPARK-19495-2.
* [MINOR][DOCS] Fix typo in Hive ExamplesDustin Koupal2017-04-063-3/+3
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix typo in hive examples from "DaraFrames" to "DataFrames" ## How was this patch tested? N/A Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Dustin Koupal <dkoupal@blizzard.com> Closes #17554 from cooper6581/typo-daraframes.
* [SPARK-17019][CORE] Expose on-heap and off-heap memory usage in various placesjerryshao2017-04-0623-119/+638
| | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? With [SPARK-13992](https://issues.apache.org/jira/browse/SPARK-13992), Spark supports persisting data into off-heap memory, but the usage of on-heap and off-heap memory is not exposed currently, it is not so convenient for user to monitor and profile, so here propose to expose off-heap memory as well as on-heap memory usage in various places: 1. Spark UI's executor page will display both on-heap and off-heap memory usage. 2. REST request returns both on-heap and off-heap memory. 3. Also this can be gotten from MetricsSystem. 4. Last this usage can be obtained programmatically from SparkListener. Attach the UI changes: ![screen shot 2016-08-12 at 11 20 44 am](https://cloud.githubusercontent.com/assets/850797/17612032/6c2f4480-607f-11e6-82e8-a27fb8cbb4ae.png) Backward compatibility is also considered for event-log and REST API. Old event log can still be replayed with off-heap usage displayed as 0. For REST API, only adds the new fields, so JSON backward compatibility can still be kept. ## How was this patch tested? Unit test added and manual verification. Author: jerryshao <sshao@hortonworks.com> Closes #14617 from jerryshao/SPARK-17019.
* [SPARK-20195][SPARKR][SQL] add createTable catalog API and deprecate ↵Felix Cheung2017-04-064-16/+68
| | | | | | | | | | | | | | | | createExternalTable ## What changes were proposed in this pull request? Following up on #17483, add createTable (which is new in 2.2.0) and deprecate createExternalTable, plus a number of minor fixes ## How was this patch tested? manual, unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17511 from felixcheung/rceatetable.
* [SPARK-20196][PYTHON][SQL] update doc for catalog functions for all ↵Felix Cheung2017-04-066-52/+79
| | | | | | | | | | | | | | | | languages, add pyspark refreshByPath API ## What changes were proposed in this pull request? Update doc to remove external for createTable, add refreshByPath in python ## How was this patch tested? manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17512 from felixcheung/catalogdoc.
* [SPARK-20064][PYSPARK] Bump the PySpark verison number to 2.2setjet2017-04-061-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? PySpark version in version.py was lagging behind Versioning is in line with PEP 440: https://www.python.org/dev/peps/pep-0440/ ## How was this patch tested? Simply rebuild the project with existing tests Author: setjet <rubenljanssen@gmail.com> Author: Ruben Janssen <rubenljanssen@gmail.com> Closes #17523 from setjet/SPARK-20064.
* [SPARK-20085][MESOS] Configurable mesos labels for executorsKalvin Chau2017-04-063-0/+79
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add spark.mesos.task.labels configuration option to add mesos key:value labels to the executor. "k1:v1,k2:v2" as the format, colons separating key-value and commas to list out more than one. Discussion of labels with mgummelt at #17404 ## How was this patch tested? Added unit tests to verify labels were added correctly, with incorrect labels being ignored and added a test to test the name of the executor. Tested with: `./build/sbt -Pmesos mesos/test` Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Kalvin Chau <kalvin.chau@viasat.com> Closes #17413 from kalvinnchau/mesos-labels.
* [SPARK-19953][ML] Random Forest Models use parent UID when being fitBryan Cutler2017-04-0641-100/+98
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The ML `RandomForestClassificationModel` and `RandomForestRegressionModel` were not using the estimator parent UID when being fit. This change fixes that so the models can be properly be identified with their parents. ## How was this patch tested?Existing tests. Added check to verify that model uid matches that of the parent, then renamed `checkCopy` to `checkCopyAndUids` and verified that it was called by one test for each ML algorithm. Author: Bryan Cutler <cutlerb@gmail.com> Closes #17296 from BryanCutler/rfmodels-use-parent-uid-SPARK-19953.
* [SPARK-20217][CORE] Executor should not fail stage if killed task throws ↵Eric Liang2017-04-052-2/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | non-interrupted exception ## What changes were proposed in this pull request? If tasks throw non-interrupted exceptions on kill (e.g. java.nio.channels.ClosedByInterruptException), their death is reported back as TaskFailed instead of TaskKilled. This causes stage failure in some cases. This is reproducible as follows. Run the following, and then use SparkContext.killTaskAttempt to kill one of the tasks. The entire stage will fail since we threw a RuntimeException instead of InterruptedException. ``` spark.range(100).repartition(100).foreach { i => try { Thread.sleep(10000000) } catch { case t: InterruptedException => throw new RuntimeException(t) } } ``` Based on the code in TaskSetManager, I think this also affects kills of speculative tasks. However, since the number of speculated tasks is few, and usually you need to fail a task a few times before the stage is cancelled, it unlikely this would be noticed in production unless both speculation was enabled and the num allowed task failures was = 1. We should probably unconditionally return TaskKilled instead of TaskFailed if the task was killed by the driver, regardless of the actual exception thrown. ## How was this patch tested? Unit test. The test fails before the change in Executor.scala cc JoshRosen Author: Eric Liang <ekl@databricks.com> Closes #17531 from ericl/fix-task-interrupt.
* [SPARK-20231][SQL] Refactor star schema code for the subsequent star join ↵Ioana Delaney2017-04-053-329/+354
| | | | | | | | | | | | | | | detection in CBO ## What changes were proposed in this pull request? This commit moves star schema code from ```join.scala``` to ```StarSchemaDetection.scala```. It also applies some minor fixes in ```StarJoinReorderSuite.scala```. ## How was this patch tested? Run existing ```StarJoinReorderSuite.scala```. Author: Ioana Delaney <ioanamdelaney@gmail.com> Closes #17544 from ioana-delaney/starSchemaCBOv2.
* [SPARK-20214][ML] Make sure converted csc matrix has sorted indicesLiang-Chi Hsieh2017-04-053-0/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `_convert_to_vector` converts a scipy sparse matrix to csc matrix for initializing `SparseVector`. However, it doesn't guarantee the converted csc matrix has sorted indices and so a failure happens when you do something like that: from scipy.sparse import lil_matrix lil = lil_matrix((4, 1)) lil[1, 0] = 1 lil[3, 0] = 2 _convert_to_vector(lil.todok()) File "/home/jenkins/workspace/python/pyspark/mllib/linalg/__init__.py", line 78, in _convert_to_vector return SparseVector(l.shape[0], csc.indices, csc.data) File "/home/jenkins/workspace/python/pyspark/mllib/linalg/__init__.py", line 556, in __init__ % (self.indices[i], self.indices[i + 1])) TypeError: Indices 3 and 1 are not strictly increasing A simple test can confirm that `dok_matrix.tocsc()` won't guarantee sorted indices: >>> from scipy.sparse import lil_matrix >>> lil = lil_matrix((4, 1)) >>> lil[1, 0] = 1 >>> lil[3, 0] = 2 >>> dok = lil.todok() >>> csc = dok.tocsc() >>> csc.has_sorted_indices 0 >>> csc.indices array([3, 1], dtype=int32) I checked the source codes of scipy. The only way to guarantee it is `csc_matrix.tocsr()` and `csr_matrix.tocsc()`. ## How was this patch tested? Existing tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #17532 from viirya/make-sure-sorted-indices.
* [SPARK-20204][SQL][FOLLOWUP] SQLConf should react to change in default ↵Dilip Biswal2017-04-064-1/+35
| | | | | | | | | | | | | | timezone settings ## What changes were proposed in this pull request? Make sure SESSION_LOCAL_TIMEZONE reflects the change in JVM's default timezone setting. Currently several timezone related tests fail as the change to default timezone is not picked up by SQLConf. ## How was this patch tested? Added an unit test in ConfigEntrySuite Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #17537 from dilipbiswal/timezone_debug.
* [SPARK-20224][SS] Updated docs for streaming dropDuplicates and ↵Tathagata Das2017-04-055-12/+509
| | | | | | | | | | | | | | | | | | mapGroupsWithState ## What changes were proposed in this pull request? - Fixed bug in Java API not passing timeout conf to scala API - Updated markdown docs - Updated scala docs - Added scala and Java example ## How was this patch tested? Manually ran examples. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17539 from tdas/SPARK-20224.
* [SPARK-19454][PYTHON][SQL] DataFrame.replace improvementszero3232017-04-052-25/+128
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Allows skipping `value` argument if `to_replace` is a `dict`: ```python df = sc.parallelize([("Alice", 1, 3.0)]).toDF() df.replace({"Alice": "Bob"}).show() ```` - Adds validation step to ensure homogeneous values / replacements. - Simplifies internal control flow. - Improves unit tests coverage. ## How was this patch tested? Existing unit tests, additional unit tests, manual testing. Author: zero323 <zero323@users.noreply.github.com> Closes #16793 from zero323/SPARK-19454.
* [SPARK-20223][SQL] Fix typo in tpcds q77.sqlwangzhenhua2017-04-051-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix typo in tpcds q77.sql ## How was this patch tested? N/A Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17538 from wzhfy/typoQ77.
* [SPARK-19807][WEB UI] Add reason for cancellation when a stage is killed ↵shaolinliu2017-04-051-1/+1
| | | | | | | | | | | | | | | | | | | using web UI ## What changes were proposed in this pull request? When a user kills a stage using web UI (in Stages page), StagesTab.handleKillRequest requests SparkContext to cancel the stage without giving a reason. SparkContext has cancelStage(stageId: Int, reason: String) that Spark could use to pass the information for monitoring/debugging purposes. ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: shaolinliu <liu.shaolin1@zte.com.cn> Author: lvdongr <lv.dongdong@zte.com.cn> Closes #17258 from shaolinliu/SPARK-19807.
* [SPARK-20042][WEB UI] Fix log page buttons for reverse proxy modeOliver Köth2017-04-051-3/+16
| | | | | | | | | | | | | | | with spark.ui.reverseProxy=true, full path URLs like /log will point to the master web endpoint which is serving the worker UI as reverse proxy. To access a REST endpoint in the worker in reverse proxy mode , the leading /proxy/"target"/ part of the base URI must be retained. Added logic to log-view.js to handle this, similar to executorspage.js Patch was tested manually Author: Oliver Köth <okoeth@de.ibm.com> Closes #17370 from okoethibm/master.
* [SPARK-20209][SS] Execute next trigger immediately if previous batch took ↵Tathagata Das2017-04-0411-72/+194
| | | | | | | | | | | | | | | | | longer than trigger interval ## What changes were proposed in this pull request? For large trigger intervals (e.g. 10 minutes), if a batch takes 11 minutes, then it will wait for 9 mins before starting the next batch. This does not make sense. The processing time based trigger policy should be to do process batches as fast as possible, but no faster than 1 in every trigger interval. If batches are taking longer than trigger interval anyways, then no point waiting extra trigger interval. In this PR, I modified the ProcessingTimeExecutor to do so. Another minor change I did was to extract our StreamManualClock into a separate class so that it can be used outside subclasses of StreamTest. For example, ProcessingTimeExecutorSuite does not need to create any context for testing, just needs the StreamManualClock. ## How was this patch tested? Added new unit tests to comprehensively test this behavior. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17525 from tdas/SPARK-20209.
* Small doc fix for ReuseSubquery.Reynold Xin2017-04-041-2/+2
|
* [SPARKR][DOC] update doc for fpgrowthFelix Cheung2017-04-042-5/+5
| | | | | | | | | | | | ## What changes were proposed in this pull request? minor update zero323 Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17526 from felixcheung/rfpgrowthfollowup.
* [SPARK-20003][ML] FPGrowthModel setMinConfidence should affect rules ↵Yuhao Yang2017-04-042-21/+56
| | | | | | | | | | | | | | | | | | | | generation and transform ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-20003 I was doing some test and found the issue. ml.fpm.FPGrowthModel `setMinConfidence` should always affect rules generation and transform. Currently associationRules in FPGrowthModel is a lazy val and `setMinConfidence` in FPGrowthModel has no impact once associationRules got computed . I try to cache the associationRules to avoid re-computation if `minConfidence` is not changed, but this makes FPGrowthModel somehow stateful. Let me know if there's any concern. ## How was this patch tested? new unit test and I strength the unit test for model save/load to ensure the cache mechanism. Author: Yuhao Yang <yuhao.yang@intel.com> Closes #17336 from hhbyyh/fpmodelminconf.
* [SPARK-20183][ML] Added outlierRatio arg to ↵Seth Hendrickson2017-04-045-6/+8
| | | | | | | | | | | | | | | | | | MLTestingUtils.testOutliersWithSmallWeights ## What changes were proposed in this pull request? This is a small piece from https://github.com/apache/spark/pull/16722 which ultimately will add sample weights to decision trees. This is to allow more flexibility in testing outliers since linear models and trees behave differently. Note: The primary author when this is committed should be sethah since this is taken from his code. ## How was this patch tested? Existing tests Author: Joseph K. Bradley <joseph@databricks.com> Closes #17501 from jkbradley/SPARK-20183.
* [SPARK-19716][SQL] support by-name resolution for struct type elements in arrayWenchen Fan2017-04-047-41/+141
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Previously when we construct deserializer expression for array type, we will first cast the corresponding field to expected array type and then apply `MapObjects`. However, by doing that, we lose the opportunity to do by-name resolution for struct type inside array type. In this PR, I introduce a `UnresolvedMapObjects` to hold the lambda function and the input array expression. Then during analysis, after the input array expression is resolved, we get the actual array element type and apply by-name resolution. Then we don't need to add `Cast` for array type when constructing the deserializer expression, as the element type is determined later at analyzer. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17398 from cloud-fan/dataset.
* [SPARK-20204][SQL] remove SimpleCatalystConf and CatalystConf type aliasWenchen Fan2017-04-0454-220/+164
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/17285 . ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #17521 from cloud-fan/conf.
* [MINOR][R] Reorder `Collate` fields in DESCRIPTION filehyukjinkwon2017-04-041-1/+1
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? It seems cran check scripts corrects `R/pkg/DESCRIPTION` and follows the order in `Collate` fields. This PR proposes to fix `catalog.R`'s order so that running this script does not show up a small diff in this file every time. ## How was this patch tested? Manually via `./R/check-cran.sh`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17528 from HyukjinKwon/minor-reorder-description.
* [SPARK-20191][YARN] Crate wrapper for RackResolver so tests can override it.Marcelo Vanzin2017-04-046-35/+56
| | | | | | | | | | | | | | Current test code tries to override the RackResolver used by setting configuration params, but because YARN libs statically initialize the resolver the first time it's used, that means that those configs don't really take effect during Spark tests. This change adds a wrapper class that easily allows tests to override the behavior of the resolver for the Spark code that uses it. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #17508 from vanzin/SPARK-20191.
* [SPARK-18278][SCHEDULER] Documentation to point to Kubernetes cluster schedulerAnirudh Ramanathan2017-04-042-1/+6
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Adding documentation to point to Kubernetes cluster scheduler being developed out-of-repo in https://github.com/apache-spark-on-k8s/spark cc rxin srowen tnachen ash211 mccheah erikerlandson ## How was this patch tested? Docs only change Author: Anirudh Ramanathan <foxish@users.noreply.github.com> Author: foxish <ramanathana@google.com> Closes #17522 from foxish/upstream-doc.
* [SPARK-20198][SQL] Remove the inconsistency in table/function name ↵Xiao Li2017-04-048-57/+186
| | | | | | | | | | | | | | | | | | | | | conventions in SparkSession.Catalog APIs ### What changes were proposed in this pull request? Observed by felixcheung , in `SparkSession`.`Catalog` APIs, we have different conventions/rules for table/function identifiers/names. Most APIs accept the qualified name (i.e., `databaseName`.`tableName` or `databaseName`.`functionName`). However, the following five APIs do not accept it. - def listColumns(tableName: String): Dataset[Column] - def getTable(tableName: String): Table - def getFunction(functionName: String): Function - def tableExists(tableName: String): Boolean - def functionExists(functionName: String): Boolean To make them consistent with the other Catalog APIs, this PR does the changes, updates the function/API comments and adds the `params` to clarify the inputs we allow. ### How was this patch tested? Added the test cases . Author: Xiao Li <gatorsmile@gmail.com> Closes #17518 from gatorsmile/tableIdentifier.
* [SPARK-20190][APP-ID] applications//jobs' in rest api,status should be ↵guoxiaolongzte2017-04-041-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | [running|s… …ucceeded|failed|unknown] ## What changes were proposed in this pull request? '/applications/[app-id]/jobs' in rest api.status should be'[running|succeeded|failed|unknown]'. now status is '[complete|succeeded|failed]'. but '/applications/[app-id]/jobs?status=complete' the server return 'HTTP ERROR 404'. Added '?status=running' and '?status=unknown'. code : public enum JobExecutionStatus { RUNNING, SUCCEEDED, FAILED, UNKNOWN; ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn> Closes #17507 from guoxiaolongzte/SPARK-20190.
* [SPARK-19825][R][ML] spark.ml R API for FPGrowthzero3232017-04-038-1/+348
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Adds SparkR API for FPGrowth: [SPARK-19825](https://issues.apache.org/jira/browse/SPARK-19825): - `spark.fpGrowth` -model training. - `freqItemsets` and `associationRules` methods with new corresponding generics. - Scala helper: `org.apache.spark.ml.r. FPGrowthWrapper` - unit tests. ## How was this patch tested? Feature specific unit tests. Author: zero323 <zero323@users.noreply.github.com> Closes #17170 from zero323/SPARK-19825.
* [SPARK-20067][SQL] Unify and Clean Up Desc Commands Using Catalog InterfaceXiao Li2017-04-0319-737/+642
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? This PR is to unify and clean up the outputs of `DESC EXTENDED/FORMATTED` and `SHOW TABLE EXTENDED` by moving the logics into the Catalog interface. The output formats are improved. We also add the missing attributes. It impacts the DDL commands like `SHOW TABLE EXTENDED`, `DESC EXTENDED` and `DESC FORMATTED`. In addition, by following what we did in Dataset API `printSchema`, we can use `treeString` to show the schema in the more readable way. Below is the current way: ``` Schema: STRUCT<`a`: STRING (nullable = true), `b`: INT (nullable = true), `c`: STRING (nullable = true), `d`: STRING (nullable = true)> ``` After the change, it should look like ``` Schema: root |-- a: string (nullable = true) |-- b: integer (nullable = true) |-- c: string (nullable = true) |-- d: string (nullable = true) ``` ### How was this patch tested? `describe.sql` and `show-tables.sql` Author: Xiao Li <gatorsmile@gmail.com> Closes #17394 from gatorsmile/descFollowUp.
* [SPARK-10364][SQL] Support Parquet logical type TIMESTAMP_MILLISDilip Biswal2017-04-0412-20/+221
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? **Description** from JIRA The TimestampType in Spark SQL is of microsecond precision. Ideally, we should convert Spark SQL timestamp values into Parquet TIMESTAMP_MICROS. But unfortunately parquet-mr hasn't supported it yet. For the read path, we should be able to read TIMESTAMP_MILLIS Parquet values and pad a 0 microsecond part to read values. For the write path, currently we are writing timestamps as INT96, similar to Impala and Hive. One alternative is that, we can have a separate SQL option to let users be able to write Spark SQL timestamp values as TIMESTAMP_MILLIS. Of course, in this way the microsecond part will be truncated. ## How was this patch tested? Added new tests in ParquetQuerySuite and ParquetIOSuite Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #15332 from dilipbiswal/parquet-time-millis.
* [SPARK-19408][SQL] filter estimation on two columns of same tableRon Hu2017-04-032-10/+363
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In SQL queries, we also see predicate expressions involving two columns such as "column-1 (op) column-2" where column-1 and column-2 belong to same table. Note that, if column-1 and column-2 belong to different tables, then it is a join operator's work, NOT a filter operator's work. This PR estimates filter selectivity on two columns of same table. For example, multiple tpc-h queries have this predicate "WHERE l_commitdate < l_receiptdate" ## How was this patch tested? We added 6 new test cases to test various logical predicates involving two columns of same table. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Ron Hu <ron.hu@huawei.com> Author: U-CHINA\r00754707 <r00754707@R00754707-SC04.china.huawei.com> Closes #17415 from ron8hu/filterTwoColumns.
* [SPARK-20145] Fix range case insensitive bug in SQLsamelamin2017-04-033-4/+26
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Range in SQL should be case insensitive ## How was this patch tested? unit test Author: samelamin <hussam.elamin@gmail.com> Author: samelamin <sam_elamin@discovery.com> Closes #17487 from samelamin/SPARK-20145.
* [SPARK-20194] Add support for partition pruning to in-memory catalogAdrian Ionescu2017-04-037-45/+85
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch implements `listPartitionsByFilter()` for `InMemoryCatalog` and thus resolves an outstanding TODO causing the `PruneFileSourcePartitions` optimizer rule not to apply when "spark.sql.catalogImplementation" is set to "in-memory" (which is the default). The change is straightforward: it extracts the code for further filtering of the list of partitions returned by the metastore's `getPartitionsByFilter()` out from `HiveExternalCatalog` into `ExternalCatalogUtils` and calls this new function from `InMemoryCatalog` on the whole list of partitions. Now that this method is implemented we can always pass the `CatalogTable` to the `DataSource` in `FindDataSourceTable`, so that the latter is resolved to a relation with a `CatalogFileIndex`, which is what the `PruneFileSourcePartitions` rule matches for. ## How was this patch tested? Ran existing tests and added new test for `listPartitionsByFilter` in `ExternalCatalogSuite`, which is subclassed by both `InMemoryCatalogSuite` and `HiveExternalCatalogSuite`. Author: Adrian Ionescu <adrian@databricks.com> Closes #17510 from adrian-ionescu/InMemoryCatalog.
* [SPARK-19641][SQL] JSON schema inference in DROPMALFORMED mode produces ↵hyukjinkwon2017-04-032-33/+78
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | incorrect schema for non-array/object JSONs ## What changes were proposed in this pull request? Currently, when we infer the types for vaild JSON strings but object or array, we are producing empty schemas regardless of parse modes as below: ```scala scala> spark.read.option("mode", "DROPMALFORMED").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema() root ``` ```scala scala> spark.read.option("mode", "FAILFAST").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema() root ``` This PR proposes to handle parse modes in type inference. After this PR, ```scala scala> spark.read.option("mode", "DROPMALFORMED").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema() root |-- a: long (nullable = true) ``` ``` scala> spark.read.option("mode", "FAILFAST").json(Seq("""{"a": 1}""", """"a"""").toDS).printSchema() java.lang.RuntimeException: Failed to infer a common schema. Struct types are expected but string was found. ``` This PR is based on https://github.com/NathanHowell/spark/commit/e233fd03346a73b3b447fa4c24f3b12c8b2e53ae and I and NathanHowell talked about this in https://issues.apache.org/jira/browse/SPARK-19641 ## How was this patch tested? Unit tests in `JsonSuite` for both `DROPMALFORMED` and `FAILFAST` modes. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17492 from HyukjinKwon/SPARK-19641.
* [SPARK-19969][ML] Imputer doc and exampleYuhao Yang2017-04-035-1/+244
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add docs and examples for spark.ml.feature.Imputer. Currently scala and Java examples are included. Python example will be added after https://github.com/apache/spark/pull/17316 ## How was this patch tested? local doc generation and example execution Author: Yuhao Yang <yuhao.yang@intel.com> Closes #17324 from hhbyyh/imputerdoc.