Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
154 commits
Select commit Hold shift + click to select a range
c87f26b
add RowBatch and RowBasedHashMapGenerator
ooq Jul 13, 2016
772c213
[SPARK-16303][DOCS][EXAMPLES] Updated SQL programming guide and examples
Jul 13, 2016
45641e5
fix scale codestyle
ooq Jul 13, 2016
c190d89
[SPARK-15889][STREAMING] Follow-up fix to erroneous condition in Stre…
srowen Jul 13, 2016
f156136
[SPARK-16375][WEB UI] Fixed misassigned var: numCompletedTasks was as…
ajbozarth Jul 13, 2016
f73891e
[MINOR] Fix Java style errors and remove unused imports
keypointt Jul 13, 2016
83879eb
[SPARK-16439] Fix number formatting in SQL UI
Jul 13, 2016
bf107f1
[SPARK-16438] Add Asynchronous Actions documentation
phalodi Jul 13, 2016
3d6f679
[MINOR][YARN] Fix code error in yarn-cluster unit test
sharkdtu Jul 13, 2016
51ade51
[SPARK-16440][MLLIB] Undeleted broadcast variables in Word2Vec causin…
srowen Jul 13, 2016
ea06e4e
[SPARK-16469] enhanced simulate multiply
uzadude Jul 13, 2016
f376c37
[SPARK-16343][SQL] Improve the PushDownPredicate rule to pushdown pre…
jiangxb1987 Jul 13, 2016
d8220c1
[SPARK-16435][YARN][MINOR] Add warning log if initialExecutors is les…
jerryshao Jul 13, 2016
01f09b1
[SPARK-14812][ML][MLLIB][PYTHON] Experimental, DeveloperApi annotatio…
jkbradley Jul 13, 2016
0744d84
[SPARK-16531][SQL][TEST] Remove timezone setting from DataFrameTimeWi…
brkyvz Jul 13, 2016
51a6706
[SPARK-16114][SQL] updated structured streaming guide
jjthomas Jul 13, 2016
b4baf08
[SPARKR][MINOR] R examples and test updates
felixcheung Jul 13, 2016
fb2e8ee
[SPARKR][DOCS][MINOR] R programming guide to include csv data source …
felixcheung Jul 13, 2016
c5ec879
[SPARK-16482][SQL] Describe Table Command for Tables Requiring Runtim…
gatorsmile Jul 13, 2016
a5f51e2
[SPARK-16485][ML][DOC] Fix privacy of GLM members, rename sqlDataType…
jkbradley Jul 13, 2016
9c53057
[SPARK-16536][SQL][PYSPARK][MINOR] Expose `sql` in PySpark Shell
dongjoon-hyun Jul 14, 2016
39c836e
[SPARK-16503] SparkSession should provide Spark version
lw-lin Jul 14, 2016
db7317a
[SPARK-16448] RemoveAliasOnlyProject should not remove alias with met…
cloud-fan Jul 14, 2016
252d4f2
[SPARK-16500][ML][MLLIB][OPTIMIZER] add LBFGS convergence warning for…
WeichenXu123 Jul 14, 2016
e3f8a03
[SPARK-16403][EXAMPLES] Cleanup to remove unused imports, consistent …
BryanCutler Jul 14, 2016
c4bc2ed
[SPARK-14963][MINOR][YARN] Fix typo in YarnShuffleService recovery fi…
jerryshao Jul 14, 2016
b7b5e17
[SPARK-16505][YARN] Optionally propagate error during shuffle service…
Jul 14, 2016
1b5c9e5
[SPARK-16530][SQL][TRIVIAL] Wrong Parser Keyword in ALTER TABLE CHANG…
gatorsmile Jul 14, 2016
56183b8
[SPARK-16543][SQL] Rename the columns of `SHOW PARTITION/COLUMNS` com…
dongjoon-hyun Jul 14, 2016
093ebbc
[SPARK-16509][SPARKR] Rename window.partitionBy and window.orderBy to…
sun-rui Jul 14, 2016
12005c8
[SPARK-16538][SPARKR] fix R call with namespace operator on SparkSess…
felixcheung Jul 14, 2016
c576f9f
[SPARK-16529][SQL][TEST] `withTempDatabase` should set `default` data…
dongjoon-hyun Jul 14, 2016
31ca741
[SPARK-16528][SQL] Fix NPE problem in HiveClientImpl
jacek-lewandowski Jul 14, 2016
91575ca
[SPARK-16540][YARN][CORE] Avoid adding jars twice for Spark running o…
jerryshao Jul 14, 2016
01c4c1f
[SPARK-16553][DOCS] Fix SQL example file name in docs
shivaram Jul 14, 2016
972673a
[SPARK-16555] Work around Jekyll error-handling bug which led to sile…
JoshRosen Jul 14, 2016
2e4075e
[SPARK-16557][SQL] Remove stale doc in sql/README.md
rxin Jul 15, 2016
1832423
[SPARK-16546][SQL][PYSPARK] update python dataframe.drop
WeichenXu123 Jul 15, 2016
71ad945
[SPARK-16426][MLLIB] Fix bug that caused NaNs in IsotonicRegression
neggert Jul 15, 2016
22d8afd
update data structures to be consistent with what is used
ooq Jul 15, 2016
5ffd5d3
[SPARK-14817][ML][MLLIB][DOC] Made DataFrame-based API primary in MLl…
jkbradley Jul 15, 2016
611a8ca
[SPARK-16538][SPARKR] Add more tests for namespace call to SparkSessi…
felixcheung Jul 15, 2016
b2f24f9
[SPARK-16230][CORE] CoarseGrainedExecutorBackend to self kill if ther…
tejasapatil Jul 15, 2016
a1ffbad
[SPARK-16582][SQL] Explicitly define isNull = false for non-nullable …
sameeragarwal Jul 16, 2016
5ec0d69
[SPARK-3359][DOCS] More changes to resolve javadoc 8 errors that will…
srowen Jul 16, 2016
4167304
[SPARK-16112][SPARKR] Programming guide for gapply/gapplyCollect
Jul 16, 2016
c33e4b0
[SPARK-16507][SPARKR] Add a CRAN checker, fix Rd aliases
shivaram Jul 17, 2016
7b84758
[SPARK-16584][SQL] Move regexp unit tests to RegexpExpressionsSuite
rxin Jul 17, 2016
33b2910
add simplerowbatch
ooq Jul 17, 2016
d27fe9b
[SPARK-16027][SPARKR] Fix R tests SparkSession init/stop
felixcheung Jul 18, 2016
480c870
[SPARK-16588][SQL] Deprecate monotonicallyIncreasingId in Scala/Java
rxin Jul 18, 2016
2c1973a
Add tests for SimpleRowBatch
ooq Jul 18, 2016
a529fc9
[MINOR][TYPO] fix fininsh typo
WeichenXu123 Jul 18, 2016
8ea3f4e
[SPARK-16055][SPARKR] warning added while using sparkPackages with sp…
krishnakalyan3 Jul 18, 2016
2877f1a
[SPARK-16351][SQL] Avoid per-record type dispatch in JSON when writing
HyukjinKwon Jul 18, 2016
8f538b1
Clean logic in SimpleRowBatch that was supposedly to deal with multip…
ooq Jul 18, 2016
96e9afa
[SPARK-16515][SQL] set default record reader and writer for script tr…
adrian-wang Jul 18, 2016
d939488
remove Rowbatch; renaming SimpleRowBatch to RowBasedKeyValueBatch
ooq Jul 18, 2016
b04ff88
fix indent
ooq Jul 18, 2016
75f0efe
[SPARKR][DOCS] minor code sample update in R programming guide
felixcheung Jul 18, 2016
ea78edb
[SPARK-16590][SQL] Improve LogicalPlanToSQLSuite to check generated S…
dongjoon-hyun Jul 19, 2016
c4524f5
[HOTFIX] Fix Scala 2.10 compilation
rxin Jul 19, 2016
69c7730
[SPARK-16615][SQL] Expose sqlContext in SparkSession
rxin Jul 19, 2016
e5fbb18
[MINOR] Remove unused arg in als.py
zhengruifeng Jul 19, 2016
1426a08
[SPARK-16303][DOCS][EXAMPLES] Minor Scala/Java example update
liancheng Jul 19, 2016
6ee40d2
[DOC] improve python doc for rdd.histogram and dataframe.join
mortada Jul 19, 2016
556a943
[MINOR][BUILD] Fix Java Linter `LineLength` errors
dongjoon-hyun Jul 19, 2016
21a6dd2
[SPARK-16535][BUILD] In pom.xml, remove groupId which is redundant de…
keypointt Jul 19, 2016
6caa220
[MINOR][SQL][STREAMING][DOCS] Fix minor typos, punctuations and grammar
ahmed-mahran Jul 19, 2016
8310c07
[SPARK-16600][MLLIB] fix some latex formula syntax error
WeichenXu123 Jul 19, 2016
6c4b9f4
[SPARK-16395][STREAMING] Fail if too many CheckpointWriteHandlers are…
srowen Jul 19, 2016
5d92326
[SPARK-16478] graphX (added graph caching in strongly connected compo…
Jul 19, 2016
6708914
[SPARK-16494][ML] Upgrade breeze version to 0.12
yanboliang Jul 19, 2016
0bd76e8
[SPARK-16620][CORE] Add back the tokenization process in `RDD.pipe(co…
lw-lin Jul 19, 2016
162d04a
[SPARK-16602][SQL] `Nvl` function should support numeric-string cases
dongjoon-hyun Jul 19, 2016
1354f91
Fixs all comments except re-structuring
ooq Jul 19, 2016
2ae7b88
[SPARK-15705][SQL] Change the default value of spark.sql.hive.convert…
yhuai Jul 19, 2016
e438304
Refactoring RowBasedKeyValueBatch
ooq Jul 19, 2016
b988d1d
Checking in refactored code, refactoring done
ooq Jul 19, 2016
700e53f
Refactoring HashMapGenerator
ooq Jul 19, 2016
b23d737
minor fix
ooq Jul 19, 2016
004e29c
[SPARK-14702] Make environment of SparkLauncher launched process more…
Jul 20, 2016
9674af6
[SPARK-16568][SQL][DOCUMENTATION] update sql programming guide refres…
WeichenXu123 Jul 20, 2016
fc23263
[SPARK-10683][SPARK-16510][SPARKR] Move SparkR include jar test to Sp…
shivaram Jul 20, 2016
75146be
[SPARK-16632][SQL] Respect Hive schema when merging parquet schema.
Jul 20, 2016
0dc79ff
[SPARK-16440][MLLIB] Destroy broadcasted variables even on driver
Jul 20, 2016
95abbe5
[SPARK-15923][YARN] Spark Application rest api returns 'no such app: …
weiqingy Jul 20, 2016
4b079dc
[SPARK-16613][CORE] RDD.pipe returns values for empty partitions
srowen Jul 20, 2016
b9bab4d
[SPARK-15951] Change Executors Page to use datatables to support sort…
kishorvpatil Jul 20, 2016
e3cd5b3
[SPARK-16634][SQL] Workaround JVM bug by moving some code out of ctor.
Jul 20, 2016
e651900
[SPARK-16344][SQL] Decoding Parquet array of struct with a single fie…
liancheng Jul 20, 2016
79ca51a
Remove RowBasedHashMap's inheritence from MemoryConsumer. Not throw e…
ooq Jul 21, 2016
75a06aa
[SPARK-16272][CORE] Allow config values to reference conf, env, syste…
Jul 21, 2016
cfa5ae8
[SPARK-16644][SQL] Aggregate should not propagate constraints contain…
cloud-fan Jul 21, 2016
1bf13ba
[MINOR][DOCS][STREAMING] Minor docfix schema of csv rather than parqu…
holdenk Jul 21, 2016
864b764
[SPARK-16226][SQL] Weaken JDBC isolation level to avoid locking when …
srowen Jul 21, 2016
8674054
[SPARK-16632][SQL] Use Spark requested schema to guide vectorized Par…
liancheng Jul 21, 2016
6203668
[SPARK-16640][SQL] Add codegen for Elt function
viirya Jul 21, 2016
69626ad
[SPARK-16632][SQL] Revert PR #14272: Respect Hive schema when merging…
liancheng Jul 21, 2016
235cb25
[SPARK-16194] Mesos Driver env vars
Jul 21, 2016
9abd99b
[SPARK-16656][SQL] Try to make CreateTableAsSelectSuite more stable
yhuai Jul 21, 2016
46f80a3
[SPARK-16334] Maintain single dictionary per row-batch in vectorized …
sameeragarwal Jul 21, 2016
df2c6d5
[SPARK-16287][SQL] Implement str_to_map SQL function
techaddict Jul 22, 2016
94f14b5
[SPARK-16556][SPARK-16559][SQL] Fix Two Bugs in Bucket Specification
gatorsmile Jul 22, 2016
e1bd70f
[SPARK-16287][HOTFIX][BUILD][SQL] Fix annotation argument needs to be…
jaceklaskowski Jul 22, 2016
2c72a44
[SPARK-16487][STREAMING] Fix some batches might not get marked as ful…
ahmed-mahran Jul 22, 2016
b4e16bd
[GIT] add pydev & Rstudio project file to gitignore list
WeichenXu123 Jul 22, 2016
6c56fff
[SPARK-16650] Improve documentation of spark.task.maxFailures
Jul 22, 2016
47f5b88
[SPARK-16651][PYSPARK][DOC] Make `withColumnRenamed/drop` description…
dongjoon-hyun Jul 22, 2016
e10b874
[SPARK-16622][SQL] Fix NullPointerException when the returned value o…
viirya Jul 23, 2016
bbaf568
Fix hash perf on strings.
ooq Jul 23, 2016
7131a53
Typo in comment and make two default values private to the base class.
ooq Jul 23, 2016
25db516
[SPARK-16561][MLLIB] fix multivarOnlineSummary min/max bug
WeichenXu123 Jul 23, 2016
ab6e4ae
[SPARK-16662][PYSPARK][SQL] fix HiveContext warning bug
WeichenXu123 Jul 23, 2016
86c2752
[SPARK-16690][TEST] rename SQLTestUtils.withTempTable to withTempView
cloud-fan Jul 23, 2016
53b2456
[SPARK-16380][EXAMPLES] Update SQL examples and programming guide for…
liancheng Jul 23, 2016
e3c7039
[MINOR] Close old PRs that should be closed but have not been
srowen Jul 24, 2016
d6795c7
[SPARK-16515][SQL][FOLLOW-UP] Fix test `script` on OS X/Windows...
lw-lin Jul 24, 2016
cc1d2dc
[SPARK-16463][SQL] Support `truncate` option in Overwrite mode for JD…
dongjoon-hyun Jul 24, 2016
37bed97
[PYSPARK] add picklable SparseMatrix in pyspark.ml.common
WeichenXu123 Jul 24, 2016
23e047f
[SPARK-16416][CORE] force eager creation of loggers to avoid shutdown…
Jul 24, 2016
1221ce0
[SPARK-16645][SQL] rename CatalogStorageFormat.serdeProperties to pro…
cloud-fan Jul 25, 2016
daace60
[SPARK-5581][CORE] When writing sorted map output file, avoid open / …
bchocho Jul 25, 2016
468a3c3
[SPARK-16699][SQL] Fix performance bug in hash aggregate on long stri…
ooq Jul 25, 2016
68b4020
[SPARK-16648][SQL] Make ignoreNullsExpr a child expression of First a…
liancheng Jul 25, 2016
7ffd99e
[SPARK-16674][SQL] Avoid per-record type dispatch in JDBC when reading
HyukjinKwon Jul 25, 2016
d27d362
[SPARK-16660][SQL] CreateViewCommand should not take CatalogTable
cloud-fan Jul 25, 2016
64529b1
[SPARK-16691][SQL] move BucketSpec to catalyst module and use it in C…
cloud-fan Jul 25, 2016
d6a5217
[SPARK-16668][TEST] Test parquet reader for row groups containing bot…
sameeragarwal Jul 25, 2016
79826f3
[SPARK-16698][SQL] Field names having dots should be allowed for data…
HyukjinKwon Jul 25, 2016
7ea6d28
[SPARK-16703][SQL] Remove extra whitespace in SQL generation for wind…
liancheng Jul 25, 2016
b73defd
[SPARKR][DOCS] fix broken url in doc
felixcheung Jul 25, 2016
ad3708e
[SPARK-16653][ML][OPTIMIZER] update ANN convergence tolerance param d…
WeichenXu123 Jul 25, 2016
dd784a8
[SPARK-16685] Remove audit-release scripts.
rxin Jul 25, 2016
978cd5f
[SPARK-15271][MESOS] Allow force pulling executor docker images
philipphoffmann Jul 25, 2016
3b6e1d0
[SPARK-16485][DOC][ML] Fixed several inline formatting in ml features…
lins05 Jul 25, 2016
fc17121
Revert "[SPARK-15271][MESOS] Allow force pulling executor docker images"
JoshRosen Jul 25, 2016
9349423
add RowBatch and RowBasedHashMapGenerator
ooq Jul 13, 2016
c857a10
fix scale codestyle
ooq Jul 13, 2016
bcab377
update data structures to be consistent with what is used
ooq Jul 15, 2016
9803fed
add simplerowbatch
ooq Jul 17, 2016
ee0f47c
Add tests for SimpleRowBatch
ooq Jul 18, 2016
e3260d8
Clean logic in SimpleRowBatch that was supposedly to deal with multip…
ooq Jul 18, 2016
8e6816e
remove Rowbatch; renaming SimpleRowBatch to RowBasedKeyValueBatch
ooq Jul 18, 2016
64752be
fix indent
ooq Jul 18, 2016
bc69818
Fixs all comments except re-structuring
ooq Jul 19, 2016
5c2b25a
Refactoring RowBasedKeyValueBatch
ooq Jul 19, 2016
9cf655a
Checking in refactored code, refactoring done
ooq Jul 19, 2016
1154068
Refactoring HashMapGenerator
ooq Jul 19, 2016
efe0114
minor fix
ooq Jul 19, 2016
8078ee6
Remove RowBasedHashMap's inheritence from MemoryConsumer. Not throw e…
ooq Jul 21, 2016
babf8e4
Fix hash perf on strings.
ooq Jul 23, 2016
d044fe7
Typo in comment and make two default values private to the base class.
ooq Jul 23, 2016
5bd4bca
Merge branch 'rowbasedfastaggmap-pr1' of github.com:ooq/spark into ro…
ooq Jul 25, 2016
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
The table of contents is too big for display.
Diff view
Diff view
  •  
  •  
  •  
4 changes: 4 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
.idea/
.idea_modules/
.project
.pydevproject
.scala_dependencies
.settings
/lib/
Expand Down Expand Up @@ -78,3 +79,6 @@ spark-warehouse/
.RData
.RHistory
.Rhistory
*.Rproj
*.Rproj.*

52 changes: 52 additions & 0 deletions R/check-cran.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
#!/bin/bash

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

set -o pipefail
set -e

FWDIR="$(cd `dirname $0`; pwd)"
pushd $FWDIR > /dev/null

if [ ! -z "$R_HOME" ]
then
R_SCRIPT_PATH="$R_HOME/bin"
else
# if system wide R_HOME is not found, then exit
if [ ! `command -v R` ]; then
echo "Cannot find 'R_HOME'. Please specify 'R_HOME' or make sure R is properly installed."
exit 1
fi
R_SCRIPT_PATH="$(dirname $(which R))"
fi
echo "USING R_HOME = $R_HOME"

# Build the latest docs
$FWDIR/create-docs.sh

# Build a zip file containing the source package
"$R_SCRIPT_PATH/"R CMD build $FWDIR/pkg

# Run check as-cran.
# TODO(shivaram): Remove the skip tests once we figure out the install mechanism

VERSION=`grep Version $FWDIR/pkg/DESCRIPTION | awk '{print $NF}'`

"$R_SCRIPT_PATH/"R CMD check --as-cran --no-tests SparkR_"$VERSION".tar.gz

popd > /dev/null
5 changes: 5 additions & 0 deletions R/pkg/.Rbuildignore
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
^.*\.Rproj$
^\.Rproj\.user$
^\.lintr$
^src-native$
^html$
8 changes: 3 additions & 5 deletions R/pkg/DESCRIPTION
Original file line number Diff line number Diff line change
@@ -1,20 +1,18 @@
Package: SparkR
Type: Package
Title: R frontend for Spark
Title: R Frontend for Apache Spark
Version: 2.0.0
Date: 2013-09-09
Date: 2016-07-07
Author: The Apache Software Foundation
Maintainer: Shivaram Venkataraman <shivaram@cs.berkeley.edu>
Imports:
methods
Depends:
R (>= 3.0),
methods,
Suggests:
testthat,
e1071,
survival
Description: R frontend for Spark
Description: The SparkR package provides an R frontend for Apache Spark.
License: Apache License (== 2.0)
Collate:
'schema.R'
Expand Down
13 changes: 11 additions & 2 deletions R/pkg/NAMESPACE
Original file line number Diff line number Diff line change
Expand Up @@ -341,5 +341,14 @@ export("partitionBy",
"rowsBetween",
"rangeBetween")

export("window.partitionBy",
"window.orderBy")
export("windowPartitionBy",
"windowOrderBy")

S3method(print, jobj)
S3method(print, structField)
S3method(print, structType)
S3method(print, summary.GeneralizedLinearRegressionModel)
S3method(structField, character)
S3method(structField, jobj)
S3method(structType, jobj)
S3method(structType, structField)
Loading