You are viewing a plain text version of this content. The canonical link for it is here.
- spark git commit: [SPARK-19378][SS] Ensure continuity of stateOperator and eventTime metrics even if there is no new data in trigger - posted by td...@apache.org on 2017/02/01 00:52:58 UTC, 1 replies.
- spark git commit: [SPARK-19163][PYTHON][SQL] Delay _judf initialization to the __call__ - posted by ho...@apache.org on 2017/02/01 02:03:48 UTC, 0 replies.
- spark git commit: [SPARK-19319][SPARKR] SparkR Kmeans summary returns error when the cluster size doesn't equal to k - posted by fe...@apache.org on 2017/02/01 05:16:42 UTC, 0 replies.
- spark git commit: [SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings in Scala/Java APIs generation - posted by sr...@apache.org on 2017/02/01 13:26:26 UTC, 0 replies.
- spark git commit: [SPARK-19410][DOC] Fix brokens links in ml-pipeline and ml-tuning - posted by sr...@apache.org on 2017/02/01 13:27:23 UTC, 1 replies.
- spark git commit: [SPARK-19296][SQL] Deduplicate url and table in JdbcUtils - posted by li...@apache.org on 2017/02/01 17:43:43 UTC, 0 replies.
- spark git commit: [SPARK-19377][WEBUI][CORE] Killed tasks should have the status as KILLED - posted by zs...@apache.org on 2017/02/01 20:55:15 UTC, 1 replies.
- spark git commit: [SPARK-19347] ReceiverSupervisorImpl can add block to ReceiverTracker multiple times because of askWithRetry. - posted by va...@apache.org on 2017/02/01 21:54:45 UTC, 0 replies.
- spark git commit: [SPARK-14352][SQL] approxQuantile should support multi columns - posted by ho...@apache.org on 2017/02/01 22:11:42 UTC, 0 replies.
- spark git commit: [SPARK-19432][CORE] Fix an unexpected failure when connecting timeout - posted by zs...@apache.org on 2017/02/02 05:39:26 UTC, 1 replies.
- spark git commit: [SPARK-19389][ML][PYTHON][DOC] Minor doc fixes for ML Python Params and LinearSVC - posted by jk...@apache.org on 2017/02/02 19:58:55 UTC, 0 replies.
- spark git commit: [SPARK-19437] Rectify spark executor id in HeartbeatReceiverSuite. - posted by zs...@apache.org on 2017/02/03 07:18:23 UTC, 0 replies.
- spark git commit: [SPARK-19411][SQL] Remove the metadata used to mark optional columns in merged Parquet schema for filter predicate pushdown - posted by rx...@apache.org on 2017/02/03 10:58:47 UTC, 0 replies.
- spark git commit: [BUILD] Close stale PRs - posted by sr...@apache.org on 2017/02/03 11:23:49 UTC, 1 replies.
- spark git commit: [SPARK-18909][SQL] The error messages in `ExpressionEncoder.toRow/fromRow` are too verbose - posted by we...@apache.org on 2017/02/03 12:27:03 UTC, 0 replies.
- spark git commit: [SPARK-19244][CORE] Sort MemoryConsumers according to their memory usage when spilling - posted by mr...@apache.org on 2017/02/03 14:14:19 UTC, 0 replies.
- spark git commit: [SPARK-19386][SPARKR][DOC] Bisecting k-means in SparkR documentation - posted by fe...@apache.org on 2017/02/03 20:19:50 UTC, 0 replies.
- spark git commit: [SPARK-19386][SPARKR][FOLLOWUP] fix error in vignettes - posted by li...@apache.org on 2017/02/04 02:02:20 UTC, 0 replies.
- spark git commit: [SPARK-10063] Follow-up: remove dead code related to an old output committer. - posted by li...@apache.org on 2017/02/04 05:12:37 UTC, 0 replies.
- spark git commit: [SPARK-19446][SQL] Remove unused findTightestCommonType in TypeCoercion - posted by li...@apache.org on 2017/02/04 06:10:25 UTC, 0 replies.
- spark git commit: [SPARK-19425][SQL] Make ExtractEquiJoinKeys support UDT columns - posted by li...@apache.org on 2017/02/04 23:58:05 UTC, 0 replies.
- spark git commit: [SPARK-19452][SPARKR] Fix bug in the name assignment method - posted by fe...@apache.org on 2017/02/05 19:38:07 UTC, 0 replies.
- spark git commit: [SPARK-19247][ML] Save large word2vec models - posted by jk...@apache.org on 2017/02/06 00:14:10 UTC, 0 replies.
- spark git commit: [SPARK-19421][ML][PYSPARK] Remove numClasses and numFeatures methods in LinearSVC - posted by ho...@apache.org on 2017/02/06 03:06:58 UTC, 0 replies.
- spark git commit: [SPARK-19279][SQL] Infer Schema for Hive Serde Tables and Block Creating a Hive Table With an Empty Schema - posted by we...@apache.org on 2017/02/06 05:30:17 UTC, 0 replies.
- spark git commit: [SPARK-19409][SPARK-17213] Cleanup Parquet workarounds/hacks due to bugs of old Parquet versions - posted by rx...@apache.org on 2017/02/06 08:10:59 UTC, 0 replies.
- spark git commit: [SPARK-17663][CORE] SchedulableBuilder should handle invalid data access via scheduler.allocation.file - posted by ir...@apache.org on 2017/02/06 14:24:36 UTC, 0 replies.
- spark git commit: [SPARK-17213][SQL][FOLLOWUP] Re-enable Parquet filter tests for binary and string - posted by we...@apache.org on 2017/02/06 15:10:17 UTC, 0 replies.
- spark git commit: [SPARK-19080][SQL] simplify data source analysis - posted by we...@apache.org on 2017/02/06 16:37:06 UTC, 0 replies.
- spark git commit: [SPARK-19398] Change one misleading log in TaskSetManager. - posted by ka...@apache.org on 2017/02/06 18:51:50 UTC, 0 replies.
- spark git commit: [SPARK-19472][SQL] Parser should not mistake CASE WHEN(...) for a function call - posted by li...@apache.org on 2017/02/06 20:28:23 UTC, 2 replies.
- spark git commit: [SPARK-19441][SQL] Remove IN type coercion from PromoteStrings - posted by we...@apache.org on 2017/02/07 01:59:24 UTC, 0 replies.
- spark git commit: [SPARK-19467][ML][PYTHON] Remove cyclic imports from pyspark.ml.pipeline - posted by jk...@apache.org on 2017/02/07 02:12:24 UTC, 0 replies.
- spark git commit: [SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it from uri scheme - posted by zs...@apache.org on 2017/02/07 05:03:26 UTC, 1 replies.
- spark git commit: [SPARK-18967][SCHEDULER] compute locality levels even if delay = 0 - posted by ka...@apache.org on 2017/02/07 06:38:16 UTC, 0 replies.
- spark git commit: [SPARK-18601][SQL] Simplify Create/Get complex expression pairs in optimizer - posted by hv...@apache.org on 2017/02/07 09:55:03 UTC, 0 replies.
- spark git commit: [SPARK-19444][ML][DOCUMENTATION] Fix imports not being present in documentation - posted by sr...@apache.org on 2017/02/07 11:44:22 UTC, 1 replies.
- spark git commit: [SPARK-19260] Spaces or "%20" in path parameter are not correctly handled with… - posted by sr...@apache.org on 2017/02/07 12:21:38 UTC, 0 replies.
- spark git commit: [SPARK-16101][SQL] Refactoring CSV schema inference path to be consistent with JSON - posted by we...@apache.org on 2017/02/07 13:02:30 UTC, 0 replies.
- spark git commit: [SPARK-19118][SQL] Percentile support for frequency distribution table - posted by hv...@apache.org on 2017/02/07 13:05:29 UTC, 0 replies.
- spark git commit: [SPARK-19447] Fixing input metrics for range operator. - posted by rx...@apache.org on 2017/02/07 13:21:35 UTC, 0 replies.
- spark git commit: [SPARK-16609] Add to_date/to_timestamp with format functions - posted by hv...@apache.org on 2017/02/07 14:50:41 UTC, 0 replies.
- spark git commit: [SPARK-19495][SQL] Make SQLConf slightly more extensible - posted by rx...@apache.org on 2017/02/07 17:55:25 UTC, 0 replies.
- spark git commit: [SPARK-18609][SPARK-18841][SQL] Fix redundant Alias removal in the optimizer - posted by hv...@apache.org on 2017/02/07 21:29:08 UTC, 0 replies.
- [1/2] spark git commit: [SPARK-18682][SS] Batch Source for Kafka - posted by zs...@apache.org on 2017/02/07 22:31:27 UTC, 1 replies.
- [2/2] spark git commit: [SPARK-18682][SS] Batch Source for Kafka - posted by zs...@apache.org on 2017/02/07 22:31:28 UTC, 1 replies.
- spark git commit: [SPARK-19397][SQL] Make option names of LIBSVM and TEXT case insensitive - posted by we...@apache.org on 2017/02/08 01:33:27 UTC, 0 replies.
- spark git commit: [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operations - posted by zs...@apache.org on 2017/02/08 04:21:05 UTC, 0 replies.
- spark git commit: [SPARK-19499][SS] Add more notes in the comments of Sink.addBatch() - posted by zs...@apache.org on 2017/02/08 04:25:20 UTC, 1 replies.
- spark git commit: [SPARK-18873][SQL][TEST] New test cases for scalar subquery (part 1 of 2) - scalar subquery in SELECT clause - posted by li...@apache.org on 2017/02/08 04:36:42 UTC, 0 replies.
- spark git commit: [MINOR][DOC] Remove parenthesis in readStream() on kafka structured streaming doc - posted by zs...@apache.org on 2017/02/08 05:46:22 UTC, 1 replies.
- spark git commit: [SPARK-19488][SQL] fix csv infer schema when the field is Nan/Inf etc - posted by we...@apache.org on 2017/02/08 06:30:53 UTC, 0 replies.
- spark git commit: [SPARK-18609][SPARK-18841][SQL][BACKPORT-2.1] Fix redundant Alias removal in the optimizer - posted by hv...@apache.org on 2017/02/08 07:35:21 UTC, 0 replies.
- [1/2] spark git commit: [SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and earlier - posted by sr...@apache.org on 2017/02/08 12:20:14 UTC, 0 replies.
- [2/2] spark git commit: [SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and earlier - posted by sr...@apache.org on 2017/02/08 12:20:15 UTC, 0 replies.
- spark git commit: [SPARK-19409][BUILD][TEST-MAVEN] Fix ParquetAvroCompatibilitySuite failure due to test dependency on avro - posted by sr...@apache.org on 2017/02/08 12:21:57 UTC, 0 replies.
- spark git commit: [SPARK-19279][SQL][FOLLOW-UP] Infer Schema for Hive Serde Tables - posted by li...@apache.org on 2017/02/08 15:12:01 UTC, 0 replies.
- spark git commit: [SPARK-19464][BUILD][HOTFIX][TEST-HADOOP2.6] Add back mockito test dep in YARN module, as it ends up being required in a Maven build - posted by sr...@apache.org on 2017/02/08 17:27:25 UTC, 0 replies.
- spark git commit: [SPARK-19400][ML] Allow GLM to handle intercept only model - posted by jk...@apache.org on 2017/02/08 18:42:25 UTC, 0 replies.
- spark git commit: [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operations for branch-2.1 - posted by zs...@apache.org on 2017/02/08 19:34:04 UTC, 0 replies.
- spark git commit: [SPARK-19464][BUILD][HOTFIX] run-tests should use hadoop2.6 - posted by sr...@apache.org on 2017/02/08 21:28:10 UTC, 0 replies.
- spark git commit: [SPARK-18872][SQL][TESTS] New test cases for EXISTS subquery (Aggregate, Having, Orderby, Limit) - posted by li...@apache.org on 2017/02/09 05:32:01 UTC, 0 replies.
- spark git commit: [SPARK-19359][SQL] renaming partition should not leave useless directories - posted by li...@apache.org on 2017/02/09 05:39:30 UTC, 0 replies.
- spark git commit: [SPARK-19265][SQL][FOLLOW-UP] Configurable `tableRelationCache` maximum size - posted by li...@apache.org on 2017/02/09 05:49:01 UTC, 0 replies.
- spark git commit: [MINOR][CORE] Fix incorrect documentation of WritableConverter - posted by sr...@apache.org on 2017/02/09 11:18:07 UTC, 0 replies.
- spark git commit: [SPARK-17874][CORE] Add SSL port configuration. - posted by sa...@apache.org on 2017/02/09 13:07:04 UTC, 0 replies.
- spark git commit: [SPARK-19514] Making range interruptible. - posted by rx...@apache.org on 2017/02/09 18:07:13 UTC, 0 replies.
- spark git commit: [SPARK-19512][SQL] codegen for compare structs fails - posted by hv...@apache.org on 2017/02/09 18:15:19 UTC, 0 replies.
- spark git commit: [SPARK-19025][SQL] Remove SQL builder for operators - posted by hv...@apache.org on 2017/02/09 18:35:48 UTC, 0 replies.
- spark git commit: [SPARK-16554][CORE] Automatically Kill Executors and Nodes when they are Blacklisted - posted by ir...@apache.org on 2017/02/09 18:49:43 UTC, 0 replies.
- spark git commit: [SPARK-19481] [REPL] [MAVEN] Avoid to leak SparkContext in Signaling.cancelOnInterrupt - posted by da...@apache.org on 2017/02/09 19:14:59 UTC, 1 replies.
- spark git commit: [SPARK-19509][SQL] Grouping Sets do not respect nullable grouping columns - posted by hv...@apache.org on 2017/02/09 20:01:32 UTC, 1 replies.
- spark git commit: [SPARK-19263] Fix race in SchedulerIntegrationSuite. - posted by ka...@apache.org on 2017/02/10 00:05:59 UTC, 0 replies.
- spark git commit: [SPARK-19512][BACKPORT-2.1][SQL] codegen for compare structs fails #16852 - posted by rx...@apache.org on 2017/02/10 09:50:11 UTC, 0 replies.
- spark git commit: [SPARK-19543] from_json fails when the input row is empty - posted by hv...@apache.org on 2017/02/10 11:55:17 UTC, 1 replies.
- spark git commit: [SPARK-19545][YARN] Fix compile issue for Spark on Yarn when building against Hadoop 2.6.0~2.6.3 - posted by sr...@apache.org on 2017/02/10 13:44:33 UTC, 0 replies.
- spark git commit: [SPARK-10748][MESOS] Log error instead of crashing Spark Mesos dispatcher when a job is misconfigured - posted by sr...@apache.org on 2017/02/10 14:11:59 UTC, 0 replies.
- spark git commit: Encryption of shuffle files - posted by rx...@apache.org on 2017/02/10 16:11:08 UTC, 0 replies.
- spark git commit: [SPARK-19466][CORE][SCHEDULER] Improve Fair Scheduler Logging - posted by ka...@apache.org on 2017/02/10 16:33:03 UTC, 0 replies.
- spark git commit: [SPARK-19459][SQL] Add Hive datatype (char/varchar) to StructField metadata - posted by we...@apache.org on 2017/02/10 19:07:05 UTC, 0 replies.
- spark git commit: [SPARK-18613][ML] make spark.mllib LDA dependencies in spark.ml LDA private - posted by jk...@apache.org on 2017/02/10 19:50:27 UTC, 0 replies.
- spark git commit: [SPARK-19549] Allow providing reason for stage/job cancelling - posted by rx...@apache.org on 2017/02/10 20:10:06 UTC, 0 replies.
- spark git commit: [SPARK-19548][SQL] Support Hive UDFs which return typed Lists/Maps - posted by we...@apache.org on 2017/02/10 22:47:29 UTC, 0 replies.
- spark git commit: [SPARK-18717][SQL] Make code generation for Scala Map work with immutable.Map also - posted by li...@apache.org on 2017/02/11 00:10:01 UTC, 0 replies.
- spark git commit: [SPARK-19537] Move pendingPartitions to ShuffleMapStage. - posted by ka...@apache.org on 2017/02/11 06:35:35 UTC, 0 replies.
- spark git commit: [SPARK-17897][SQL][BACKPORT-2.0] Fixed IsNotNull Constraint Inference Rule - posted by we...@apache.org on 2017/02/12 01:20:50 UTC, 0 replies.
- spark git commit: [SPARK-19448][SQL] optimize some duplication functions between HiveClientImpl and HiveUtils - posted by li...@apache.org on 2017/02/12 06:21:35 UTC, 0 replies.
- spark git commit: [SPARK-19342][SPARKR] bug fixed in collect method for collecting timestamp column - posted by fe...@apache.org on 2017/02/12 18:42:30 UTC, 1 replies.
- spark git commit: [SPARK-19319][BACKPORT-2.1][SPARKR] SparkR Kmeans summary returns error when the cluster size doesn't equal to k - posted by fe...@apache.org on 2017/02/12 18:49:36 UTC, 0 replies.
- spark git commit: [SPARK-19564][SPARK-19559][SS][KAFKA] KafkaOffsetReader's consumers should not be in the same group - posted by zs...@apache.org on 2017/02/13 07:00:25 UTC, 1 replies.
- spark git commit: [SPARK-19574][ML][DOCUMENTATION] Fix Liquid Exception: Start indices amount is not equal to end indices amount - posted by sr...@apache.org on 2017/02/13 11:18:35 UTC, 1 replies.
- spark git commit: [SPARK-19562][BUILD] Added exclude for dev/pr-deps to gitignore - posted by sr...@apache.org on 2017/02/13 11:22:35 UTC, 0 replies.
- spark git commit: [SPARK-19496][SQL] to_date udf to return null when input date is invalid - posted by hv...@apache.org on 2017/02/13 11:25:17 UTC, 0 replies.
- spark git commit: [SPARK-19544][SQL] Improve error message when some column types are compatible and others are not in set operations - posted by hv...@apache.org on 2017/02/13 15:08:37 UTC, 0 replies.
- spark git commit: [SPARK-19506][ML][PYTHON] Import warnings in pyspark.ml.util - posted by ho...@apache.org on 2017/02/13 17:26:55 UTC, 1 replies.
- spark git commit: [SPARK-19427][PYTHON][SQL] Support data type string as a returnType argument of UDF - posted by ho...@apache.org on 2017/02/13 18:37:37 UTC, 0 replies.
- spark git commit: [SPARK-19529] TransportClientFactory.createClient() shouldn't call awaitUninterruptibly() - posted by li...@apache.org on 2017/02/13 19:04:32 UTC, 2 replies.
- spark git commit: [SPARK-19514] Enhancing the test for Range interruption. - posted by rx...@apache.org on 2017/02/13 19:07:42 UTC, 0 replies.
- spark git commit: [SPARK-19542][SS] Delete the temp checkpoint if a query is stopped without errors - posted by br...@apache.org on 2017/02/13 19:55:02 UTC, 1 replies.
- spark git commit: [SPARK-17714][CORE][TEST-MAVEN][TEST-HADOOP2.6] Avoid using ExecutorClassLoader to load Netty generated classes - posted by zs...@apache.org on 2017/02/13 20:03:39 UTC, 1 replies.
- spark git commit: [HOTFIX][SPARK-19542][SS]Fix the missing import in DataStreamReaderWriterSuite - posted by zs...@apache.org on 2017/02/13 20:39:12 UTC, 0 replies.
- spark git commit: [SPARK-19435][SQL] Type coercion between ArrayTypes - posted by li...@apache.org on 2017/02/13 21:11:03 UTC, 0 replies.
- spark git commit: [SPARK-19520][STREAMING] Do not encrypt data written to the WAL. - posted by va...@apache.org on 2017/02/13 22:19:47 UTC, 1 replies.
- spark git commit: [SPARK-19429][PYTHON][SQL] Support slice arguments in Column.__getitem__ - posted by ho...@apache.org on 2017/02/13 23:24:01 UTC, 0 replies.
- spark git commit: [SPARK-19115][SQL] Supporting Create Table Like Location - posted by li...@apache.org on 2017/02/14 03:41:50 UTC, 0 replies.
- spark git commit: [SPARK-19539][SQL] Block duplicate temp table during creation - posted by li...@apache.org on 2017/02/14 03:46:02 UTC, 0 replies.
- spark git commit: [SPARK-19585][DOC][SQL] Fix the cacheTable and uncacheTable api call in the doc - posted by li...@apache.org on 2017/02/14 06:49:34 UTC, 1 replies.
- [1/2] spark git commit: [SPARK-19589][SQL] Removal of SQLGEN files - posted by hv...@apache.org on 2017/02/14 08:35:17 UTC, 0 replies.
- [2/2] spark git commit: [SPARK-19589][SQL] Removal of SQLGEN files - posted by hv...@apache.org on 2017/02/14 08:35:18 UTC, 0 replies.
- spark git commit: [SPARK-19453][PYTHON][SQL][DOC] Correct and extend DataFrame.replace docstring - posted by ho...@apache.org on 2017/02/14 17:42:33 UTC, 0 replies.
- spark git commit: [SPARK-19162][PYTHON][SQL] UserDefinedFunction should validate that func is callable - posted by ho...@apache.org on 2017/02/14 17:46:27 UTC, 0 replies.
- spark git commit: [SPARK-18541][PYTHON] Add metadata parameter to pyspark.sql.Column.alias() - posted by ho...@apache.org on 2017/02/14 17:57:53 UTC, 0 replies.
- spark git commit: [SPARK-19571][R] Fix SparkR test break on Windows via AppVeyor - posted by fe...@apache.org on 2017/02/14 19:00:45 UTC, 0 replies.
- spark git commit: [SPARK-19529][BRANCH-1.6] Backport PR #16866 to branch-1.6 - posted by jo...@apache.org on 2017/02/14 19:27:40 UTC, 0 replies.
- spark git commit: [SPARK-19501][YARN] Reduce the number of HDFS RPCs during YARN deployment - posted by va...@apache.org on 2017/02/14 19:33:38 UTC, 2 replies.
- spark git commit: [SPARK-19387][SPARKR] Tests do not run with SparkR source package in CRAN check - posted by sh...@apache.org on 2017/02/14 21:51:31 UTC, 1 replies.
- spark git commit: [SPARK-16475][SQL] Broadcast hint for SQL Queries - posted by we...@apache.org on 2017/02/14 22:11:21 UTC, 0 replies.
- spark git commit: [SPARK-19318][SQL] Fix to treat JDBC connection properties specified by the user in case-sensitive manner. - posted by we...@apache.org on 2017/02/14 23:34:15 UTC, 0 replies.
- spark git commit: [SPARK-19584][SS][DOCS] update structured streaming documentation around batch mode - posted by td...@apache.org on 2017/02/15 02:50:17 UTC, 1 replies.
- spark git commit: [SPARK-19456][SPARKR] Add LinearSVC R API - posted by fe...@apache.org on 2017/02/15 09:15:53 UTC, 0 replies.
- spark git commit: [SPARK-19607] Finding QueryExecution that matches provided executionId - posted by rx...@apache.org on 2017/02/15 16:06:08 UTC, 0 replies.
- spark git commit: [SPARK-16475][SQL] broadcast hint for SQL queries - follow up - posted by hv...@apache.org on 2017/02/15 16:10:55 UTC, 0 replies.
- spark git commit: [SPARK-19587][SQL] bucket sorting columns should not be picked from partition columns - posted by we...@apache.org on 2017/02/15 16:15:07 UTC, 0 replies.
- spark git commit: [SPARK-17076][SQL] Cardinality estimation for join based on basic column statistics - posted by we...@apache.org on 2017/02/15 16:21:55 UTC, 0 replies.
- spark git commit: [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 2nd batch - posted by hv...@apache.org on 2017/02/15 16:28:47 UTC, 0 replies.
- spark git commit: [SPARK-18873][SQL][TEST] New test cases for scalar subquery (part 2 of 2) - scalar subquery in predicate context - posted by hv...@apache.org on 2017/02/15 16:31:03 UTC, 0 replies.
- spark git commit: [SPARK-18872][SQL][TESTS] New test cases for EXISTS subquery (Joins + CTE) - posted by hv...@apache.org on 2017/02/15 16:34:10 UTC, 0 replies.
- spark git commit: [SPARK-16475][SQL] broadcast hint for SQL queries - disallow space as the delimiter - posted by rx...@apache.org on 2017/02/15 17:48:06 UTC, 0 replies.
- spark git commit: [SPARK-19590][PYSPARK][ML] Update the document for QuantileDiscretizer in pyspark - posted by ho...@apache.org on 2017/02/15 18:12:11 UTC, 0 replies.
- spark git commit: [SPARK-19160][PYTHON][SQL] Add udf decorator - posted by ho...@apache.org on 2017/02/15 18:16:38 UTC, 0 replies.
- spark git commit: [SPARK-19399][SPARKR] Add R coalesce API for DataFrame and Column - posted by fe...@apache.org on 2017/02/15 18:45:40 UTC, 1 replies.
- spark git commit: [SPARK-19331][SQL][TESTS] Improve the test coverage of SQLViewSuite - posted by we...@apache.org on 2017/02/15 18:47:18 UTC, 0 replies.
- spark git commit: [SPARK-19607][HOTFIX] Finding QueryExecution that matches provided executionId - posted by rx...@apache.org on 2017/02/15 20:57:53 UTC, 0 replies.
- spark git commit: [SPARK-19329][SQL] Reading from or writing to a datasource table with a non pre-existing location should succeed - posted by li...@apache.org on 2017/02/15 21:21:54 UTC, 0 replies.
- spark git commit: [SPARK-18937][SQL] Timezone support in CSV/JSON parsing - posted by we...@apache.org on 2017/02/15 21:26:37 UTC, 0 replies.
- spark git commit: [SPARK-19599][SS] Clean up HDFSMetadataLog - posted by zs...@apache.org on 2017/02/16 00:21:45 UTC, 1 replies.
- spark git commit: [SPARK-18080][ML][PYTHON] Python API & Examples for Locality Sensitive Hashing - posted by yl...@apache.org on 2017/02/16 00:26:08 UTC, 0 replies.
- spark git commit: [SPARK-19604][TESTS] Log the start of every Python test - posted by yh...@apache.org on 2017/02/16 02:44:01 UTC, 0 replies.
- spark git commit: [SPARK-19603][SS] Fix StreamingQuery explain command - posted by zs...@apache.org on 2017/02/16 04:51:37 UTC, 1 replies.
- spark git commit: [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 4th batch - posted by li...@apache.org on 2017/02/16 05:29:32 UTC, 0 replies.
- spark git commit: [SPARK-19399][SPARKR][BACKPORT-2.1] fix tests broken by merge - posted by fe...@apache.org on 2017/02/16 05:31:40 UTC, 0 replies.
- spark git commit: [SPARK-19618][SQL] Inconsistency wrt max. buckets allowed from Dataframe API vs SQL - posted by we...@apache.org on 2017/02/16 06:46:05 UTC, 0 replies.
- spark git commit: [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 3rd batch - posted by li...@apache.org on 2017/02/16 08:02:20 UTC, 0 replies.
- [1/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:52 UTC, 0 replies.
- [2/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:53 UTC, 0 replies.
- [3/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:54 UTC, 0 replies.
- [4/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:55 UTC, 0 replies.
- [5/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:56 UTC, 0 replies.
- [6/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:57 UTC, 0 replies.
- [7/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:58 UTC, 0 replies.
- [8/8] spark git commit: [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support - posted by sr...@apache.org on 2017/02/16 12:32:59 UTC, 0 replies.
- spark git commit: [MINOR][BUILD] Fix javadoc8 break - posted by sr...@apache.org on 2017/02/16 12:35:47 UTC, 0 replies.
- [1/2] spark-website git commit: Remove references to Java 7 and Hadoop support before 2.5 - posted by sr...@apache.org on 2017/02/16 12:38:44 UTC, 0 replies.
- [2/2] spark-website git commit: Remove references to Java 7 and Hadoop support before 2.5 - posted by sr...@apache.org on 2017/02/16 12:38:45 UTC, 0 replies.
- spark git commit: [SPARK-19436][SQL] Add missing tests for approxQuantile - posted by li...@apache.org on 2017/02/16 17:42:17 UTC, 0 replies.
- spark git commit: [SPARK-19550][HOTFIX][BUILD] Use JAVA_HOME/bin/java if JAVA_HOME is set in dev/mima - posted by sr...@apache.org on 2017/02/16 18:43:42 UTC, 0 replies.
- spark git commit: [SPARK-18352][SQL] Support parsing multiline json files - posted by we...@apache.org on 2017/02/17 04:51:24 UTC, 0 replies.
- spark git commit: [SPARK-18120][SPARK-19557][SQL] Call QueryExecutionListener callback methods for DataFrameWriter methods - posted by we...@apache.org on 2017/02/17 05:09:18 UTC, 0 replies.
- spark git commit: [MINOR][PYTHON] Fix typo docstring: 'top' -> 'topic' - posted by sr...@apache.org on 2017/02/17 15:10:33 UTC, 0 replies.
- spark git commit: [SPARK-19622][WEBUI] Fix a http error in a paged table when using a `Go` button to search. - posted by sr...@apache.org on 2017/02/17 15:11:10 UTC, 1 replies.
- spark git commit: [SPARK-19500] [SQL] Fix off-by-one bug in BytesToBytesMap - posted by da...@apache.org on 2017/02/17 17:35:38 UTC, 2 replies.
- spark git commit: [SPARK-18986][CORE] ExternalAppendOnlyMap shouldn't fail when forced to spill before calling its iterator - posted by va...@apache.org on 2017/02/17 19:28:21 UTC, 0 replies.
- spark git commit: [SPARK-19517][SS] KafkaSource fails to initialize partition offsets - posted by zs...@apache.org on 2017/02/17 19:50:38 UTC, 1 replies.
- spark git commit: [SPARK-18285][SPARKR] SparkR approxQuantile supports input multiple columns - posted by yl...@apache.org on 2017/02/17 19:58:43 UTC, 0 replies.
- spark git commit: [SPARKR][EXAMPLES] update examples to stop spark session - posted by fe...@apache.org on 2017/02/17 21:30:59 UTC, 0 replies.
- spark git commit: [SPARK-19617][SS] Fix the race condition when starting and stopping a query quickly - posted by zs...@apache.org on 2017/02/18 03:04:48 UTC, 0 replies.
- spark git commit: [SPARK-19639][SPARKR][EXAMPLE] Add spark.svmLinear example and update vignettes - posted by fe...@apache.org on 2017/02/18 05:21:15 UTC, 0 replies.
- spark git commit: [SPARK-19550] Follow-up: fixed a typo that fails the dev/make-distribution.sh script. - posted by sr...@apache.org on 2017/02/18 14:09:00 UTC, 0 replies.
- spark git commit: [MLLIB][TYPO] Replace LeastSquaresAggregator with LogisticAggregator - posted by sr...@apache.org on 2017/02/18 14:10:22 UTC, 0 replies.
- spark git commit: [SPARK-19263] DAGScheduler should avoid sending conflicting task set. - posted by ka...@apache.org on 2017/02/18 14:56:42 UTC, 0 replies.
- spark git commit: [SPARK-19447] Make Range operator generate "recordsRead" metric - posted by rx...@apache.org on 2017/02/18 15:51:43 UTC, 0 replies.
- spark git commit: [SPARK-19550][BUILD][WIP] Addendum: select Java 1.7 for scalac 2.10, still - posted by sr...@apache.org on 2017/02/19 12:24:15 UTC, 0 replies.
- spark git commit: [SPARK-19450] Replace askWithRetry with askSync. - posted by sr...@apache.org on 2017/02/19 12:34:11 UTC, 0 replies.
- [1/2] spark git commit: [SPARK-19533][EXAMPLES] Convert Java tests to use lambdas, Java 8 features - posted by sr...@apache.org on 2017/02/19 17:38:01 UTC, 0 replies.
- [2/2] spark git commit: [SPARK-19533][EXAMPLES] Convert Java tests to use lambdas, Java 8 features - posted by sr...@apache.org on 2017/02/19 17:38:02 UTC, 0 replies.
- [1/3] spark git commit: [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features - posted by sr...@apache.org on 2017/02/19 17:42:54 UTC, 0 replies.
- [2/3] spark git commit: [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features - posted by sr...@apache.org on 2017/02/19 17:42:55 UTC, 0 replies.
- [3/3] spark git commit: [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features - posted by sr...@apache.org on 2017/02/19 17:42:56 UTC, 0 replies.
- spark git commit: [SPARK-19598][SQL] Remove the alias parameter in UnresolvedRelation - posted by li...@apache.org on 2017/02/20 00:50:19 UTC, 0 replies.
- spark git commit: [SPARK-19563][SQL] avoid unnecessary sort in FileFormatWriter - posted by we...@apache.org on 2017/02/20 02:13:21 UTC, 0 replies.
- spark-website git commit: Update Java example to use Java 8; make Scala/Python pi example consistent with better Java version; minor syntax fixes to these - posted by sr...@apache.org on 2017/02/20 16:36:05 UTC, 0 replies.
- spark git commit: [SPARK-19646][CORE][STREAMING] binaryRecords replicates records in scala API - posted by sr...@apache.org on 2017/02/20 17:02:13 UTC, 2 replies.
- spark git commit: [SPARK-15453][SQL][FOLLOW-UP] FileSourceScanExec to extract `outputOrdering` information - posted by li...@apache.org on 2017/02/20 17:04:28 UTC, 0 replies.
- spark git commit: [SPARK-19646][BUILD][HOTFIX] Fix compile error from cherry-pick of SPARK-19646 into branch 2.1 - posted by sr...@apache.org on 2017/02/20 20:19:57 UTC, 0 replies.
- spark git commit: [SPARK-19669][SQL] Open up visibility for sharedState, sessionState, and a few other functions - posted by li...@apache.org on 2017/02/20 20:21:14 UTC, 0 replies.
- spark git commit: [SPARK-19669][HOTFIX][SQL] sessionState access privileges compiled failed in TestSQLContext - posted by li...@apache.org on 2017/02/21 03:20:27 UTC, 0 replies.
- spark git commit: [SPARK-19508][CORE] Improve error message when binding service fails - posted by sr...@apache.org on 2017/02/21 05:25:25 UTC, 0 replies.
- spark git commit: [SPARK-18922][TESTS] Fix new test failures on Windows due to path and resource not closed - posted by sr...@apache.org on 2017/02/21 05:26:57 UTC, 0 replies.
- spark git commit: [SPARK-19337][ML][DOC] Documentation and examples for LinearSVC - posted by fe...@apache.org on 2017/02/21 17:38:18 UTC, 0 replies.
- spark git commit: [SPARK-19626][YARN] Using the correct config to set credentials update time - posted by va...@apache.org on 2017/02/21 17:57:46 UTC, 1 replies.
- spark git commit: [SPARK-19652][UI] Do auth checks for REST API access. - posted by va...@apache.org on 2017/02/22 00:14:39 UTC, 0 replies.
- spark git commit: [SPARK-19670][SQL][TEST] Enable Bucketed Table Reading and Writing Testing Without Hive Support - posted by li...@apache.org on 2017/02/22 03:30:42 UTC, 0 replies.
- spark git commit: [SPARK-19617][SS] Fix the race condition when starting and stopping a query quickly (branch-2.1) - posted by zs...@apache.org on 2017/02/22 04:15:51 UTC, 0 replies.
- spark-website git commit: Add instructions for running individual tests. - posted by sr...@apache.org on 2017/02/22 13:50:10 UTC, 0 replies.
- spark-website git commit: Fix last update to Java pi example - posted by sr...@apache.org on 2017/02/22 13:53:11 UTC, 0 replies.
- spark git commit: [SPARK-19694][ML] Add missing 'setTopicDistributionCol' for LDAModel - posted by ml...@apache.org on 2017/02/22 14:33:23 UTC, 0 replies.
- spark git commit: [SPARK-19679][ML] Destroy broadcasted object without blocking - posted by ml...@apache.org on 2017/02/22 14:36:23 UTC, 0 replies.
- spark git commit: [SPARK-13721][SQL] Make GeneratorOuter unresolved. - posted by hv...@apache.org on 2017/02/22 14:42:50 UTC, 0 replies.
- spark git commit: [SPARK-19405][STREAMING] Support for cross-account Kinesis reads via STS - posted by br...@apache.org on 2017/02/22 16:33:10 UTC, 0 replies.
- spark git commit: [SPARK-19616][SPARKR] weightCol and aggregationDepth should be improved for some SparkR APIs - posted by fe...@apache.org on 2017/02/22 19:50:28 UTC, 0 replies.
- spark git commit: [SPARK-19666][SQL] Skip a property without getter in Java schema inference and allow empty bean in encoder creation - posted by we...@apache.org on 2017/02/22 20:42:33 UTC, 0 replies.
- spark git commit: [SPARK-19652][UI] Do auth checks for REST API access (branch-2.1). - posted by va...@apache.org on 2017/02/22 21:19:37 UTC, 0 replies.
- spark git commit: [SPARK-19554][UI, YARN] Allow SHS URL to be used for tracking in YARN RM. - posted by va...@apache.org on 2017/02/22 22:38:05 UTC, 0 replies.
- spark git commit: [SPARK-19652][UI] Do auth checks for REST API access (branch-2.0). - posted by va...@apache.org on 2017/02/22 23:49:02 UTC, 0 replies.
- spark git commit: [SPARK-19658][SQL] Set NumPartitions of RepartitionByExpression In Parser - posted by we...@apache.org on 2017/02/23 01:26:59 UTC, 0 replies.
- spark git commit: [SPARK-15615][SQL] Add an API to load DataFrame from Dataset[String] storing JSON - posted by we...@apache.org on 2017/02/23 02:03:35 UTC, 0 replies.
- spark git commit: [SPARK-16122][CORE] Add rest api for job environment - posted by va...@apache.org on 2017/02/23 04:03:07 UTC, 0 replies.
- spark git commit: [SPARK-19695][SQL] Throw an exception if a `columnNameOfCorruptRecord` field violates requirements in json formats - posted by we...@apache.org on 2017/02/23 05:39:26 UTC, 0 replies.
- spark git commit: [SPARK-19691][SQL] Fix ClassCastException when calculating percentile of decimal column - posted by hv...@apache.org on 2017/02/23 15:28:42 UTC, 0 replies.
- spark git commit: [SPARK-19459] Support for nested char/varchar fields in ORC - posted by we...@apache.org on 2017/02/23 18:25:22 UTC, 0 replies.
- spark git commit: [SPARK-19682][SPARKR] Issue warning (or error) when subset method "[[" takes vector index - posted by fe...@apache.org on 2017/02/23 19:12:06 UTC, 1 replies.
- spark git commit: [SPARK-19497][SS] Implement streaming deduplication - posted by td...@apache.org on 2017/02/23 19:25:43 UTC, 0 replies.
- spark git commit: [SPARK-18699][SQL] Put malformed tokens into a new field when parsing CSV data - posted by we...@apache.org on 2017/02/23 20:09:41 UTC, 0 replies.
- spark git commit: [SPARK-19706][PYSPARK] add Column.contains in pyspark - posted by we...@apache.org on 2017/02/23 21:22:42 UTC, 0 replies.
- spark git commit: [SPARK-19684][DOCS] Remove developer info from docs. - posted by ka...@apache.org on 2017/02/23 21:28:21 UTC, 0 replies.
- spark git commit: [SPARK-19674][SQL] Ignore driver accumulator updates don't belong to the execution when merging all accumulator updates - posted by we...@apache.org on 2017/02/23 22:31:21 UTC, 0 replies.
- spark git commit: [SPARK-19459][SQL][BRANCH-2.1] Support for nested char/varchar fields in ORC - posted by li...@apache.org on 2017/02/23 22:58:38 UTC, 0 replies.
- spark git commit: [SPARK-16122][DOCS] application environment rest api - posted by va...@apache.org on 2017/02/24 01:06:20 UTC, 0 replies.
- spark git commit: [SPARK-14772][PYTHON][ML] Fixed Params.copy method to match Scala implementation - posted by jk...@apache.org on 2017/02/24 02:06:07 UTC, 1 replies.
- spark git commit: [SPARK-17075][SQL] implemented filter estimation - posted by we...@apache.org on 2017/02/24 04:18:26 UTC, 0 replies.
- spark git commit: [SPARK-19664][SQL] put hive.metastore.warehouse.dir in hadoopconf to overwrite its original value - posted by we...@apache.org on 2017/02/24 06:57:29 UTC, 0 replies.
- spark git commit: [SPARK-19691][SQL][BRANCH-2.1] Fix ClassCastException when calculating percentile of decimal column - posted by hv...@apache.org on 2017/02/24 09:54:14 UTC, 0 replies.
- spark git commit: [SPARK-19161][PYTHON][SQL] Improving UDF Docstrings - posted by ho...@apache.org on 2017/02/24 16:22:55 UTC, 0 replies.
- spark git commit: [SPARK-19707][CORE] Improve the invalid path check for sc.addJar - posted by va...@apache.org on 2017/02/24 17:29:06 UTC, 1 replies.
- spark git commit: [SPARK-19038][YARN] Avoid overwriting keytab configuration in yarn-client - posted by va...@apache.org on 2017/02/24 17:32:12 UTC, 2 replies.
- spark git commit: [SPARK-17495][SQL] Add more tests for hive hash - posted by rx...@apache.org on 2017/02/24 17:46:46 UTC, 0 replies.
- spark git commit: [SPARK-17075][SQL] Follow up: fix file line ending and improve the tests - posted by we...@apache.org on 2017/02/24 18:24:05 UTC, 0 replies.
- spark git commit: [SPARK-17078][SQL] Show stats when explain - posted by we...@apache.org on 2017/02/24 18:25:02 UTC, 0 replies.
- spark git commit: [SPARK-19560] Improve DAGScheduler tests. - posted by ka...@apache.org on 2017/02/24 19:43:06 UTC, 0 replies.
- spark git commit: [SPARK-19597][CORE] test case for task deserialization errors - posted by ka...@apache.org on 2017/02/24 21:04:03 UTC, 0 replies.
- spark git commit: [SPARK-13330][PYSPARK] PYTHONHASHSEED is not propgated to python worker - posted by ho...@apache.org on 2017/02/24 23:04:54 UTC, 0 replies.
- spark git commit: [SPARK-15355][CORE] Proactive block replication - posted by we...@apache.org on 2017/02/24 23:40:03 UTC, 0 replies.
- spark git commit: [MINOR][DOCS] Fix few typos in structured streaming doc - posted by sr...@apache.org on 2017/02/25 02:18:27 UTC, 0 replies.
- spark git commit: [SPARK-19735][SQL] Remove HOLD_DDLTIME from Catalog APIs - posted by we...@apache.org on 2017/02/25 07:04:03 UTC, 0 replies.
- spark git commit: [SPARK-19650] Commands should not trigger a Spark job - posted by we...@apache.org on 2017/02/25 07:06:05 UTC, 0 replies.
- spark git commit: [MINOR][DOCS] Fixes two problems in the SQL programing guide page - posted by li...@apache.org on 2017/02/25 19:32:13 UTC, 1 replies.
- spark git commit: [SPARK-19673][SQL] "ThriftServer default app name is changed wrong" - posted by sr...@apache.org on 2017/02/25 21:47:07 UTC, 0 replies.
- spark git commit: [SPARK-15288][MESOS] Mesos dispatcher should handle gracefully when any thread gets UncaughtException - posted by sr...@apache.org on 2017/02/25 21:48:45 UTC, 0 replies.
- spark git commit: [MINOR][ML][DOC] Document default value for GeneralizedLinearRegression.linkPower - posted by yl...@apache.org on 2017/02/26 06:24:11 UTC, 0 replies.
- spark git commit: [SPARK-17075][SQL][FOLLOWUP] fix some minor issues and clean up the code - posted by we...@apache.org on 2017/02/26 07:01:48 UTC, 0 replies.
- spark git commit: [SQL] Duplicate test exception in SQLQueryTestSuite due to meta files(.DS_Store) on Mac - posted by li...@apache.org on 2017/02/26 07:57:01 UTC, 0 replies.
- spark git commit: [SPARK-19594][STRUCTURED STREAMING] StreamingQueryListener fails to handle QueryTerminatedEvent if more then one listeners exists - posted by zs...@apache.org on 2017/02/26 23:57:35 UTC, 1 replies.
- spark git commit: [MINOR][BUILD] Fix lint-java breaks in Java - posted by sr...@apache.org on 2017/02/27 08:44:32 UTC, 0 replies.
- spark git commit: [SPARK-15615][SQL][BUILD][FOLLOW-UP] Replace deprecated usage of json(RDD[String]) API - posted by we...@apache.org on 2017/02/27 22:33:06 UTC, 0 replies.
- spark git commit: [SPARK-19746][ML] Faster indexing for logistic aggregator - posted by db...@apache.org on 2017/02/28 00:35:09 UTC, 0 replies.
- spark git commit: [SPARK-19749][SS] Name socket source with a meaningful name - posted by zs...@apache.org on 2017/02/28 02:02:48 UTC, 0 replies.
- spark git commit: [SPARK-19748][SQL] refresh function has a wrong order to do cache invalidate and regenerate the inmemory var for InMemoryFileIndex with FileStatusCache - posted by we...@apache.org on 2017/02/28 08:16:54 UTC, 1 replies.
- [1/2] spark git commit: [SPARK-19660][CORE][SQL] Replace the configuration property names that are deprecated in the version of Hadoop 2.6 - posted by sr...@apache.org on 2017/02/28 10:13:50 UTC, 0 replies.
- [2/2] spark git commit: [SPARK-19660][CORE][SQL] Replace the configuration property names that are deprecated in the version of Hadoop 2.6 - posted by sr...@apache.org on 2017/02/28 10:13:51 UTC, 0 replies.
- spark git commit: [SPARK-14489][ML][PYSPARK] ALS unknown user/item prediction strategy - posted by ml...@apache.org on 2017/02/28 14:17:37 UTC, 0 replies.
- [1/2] spark git commit: [SPARK-19678][SQL] remove MetastoreRelation - posted by li...@apache.org on 2017/02/28 17:24:41 UTC, 0 replies.
- [2/2] spark git commit: [SPARK-19678][SQL] remove MetastoreRelation - posted by li...@apache.org on 2017/02/28 17:24:42 UTC, 0 replies.
- spark git commit: [SPARK-19677][SS] Committing a delta file atop an existing one should not fail on HDFS - posted by zs...@apache.org on 2017/02/28 18:49:31 UTC, 2 replies.
- spark git commit: [SPARK-19463][SQL] refresh cache after the InsertIntoHadoopFsRelationCommand - posted by we...@apache.org on 2017/02/28 19:59:22 UTC, 0 replies.
- spark git commit: [SPARK-19610][SQL] Support parsing multiline CSV files - posted by we...@apache.org on 2017/02/28 21:34:36 UTC, 0 replies.
- spark git commit: [MINOR][DOC] Update GLM doc to include tweedie distribution - posted by jk...@apache.org on 2017/02/28 22:43:48 UTC, 0 replies.
- spark git commit: [SPARK-19769][DOCS] Update quickstart instructions - posted by sr...@apache.org on 2017/02/28 23:07:20 UTC, 2 replies.
- spark git commit: [SPARK-19373][MESOS] Base spark.scheduler.minRegisteredResourceRatio on registered cores rather than accepted cores - posted by sr...@apache.org on 2017/02/28 23:11:03 UTC, 0 replies.
- spark git commit: [SPARK-14503][ML] spark.ml API for FPGrowth - posted by jk...@apache.org on 2017/02/28 23:53:45 UTC, 0 replies.