Skip to content

Commit

Permalink
Merge branch 'master' of github.com:apache/spark into checkpoint-cleanup
Browse files Browse the repository at this point in the history
  • Loading branch information
Andrew Or committed Jun 24, 2015
2 parents 7fb4af5 + 31f48e5 commit 5484293
Show file tree
Hide file tree
Showing 18 changed files with 552 additions and 119 deletions.
124 changes: 117 additions & 7 deletions docs/sql-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark.
All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`, `pyspark` shell, or `sparkR` shell.


## Starting Point: `SQLContext`
## Starting Point: SQLContext

<div class="codetabs">
<div data-lang="scala" markdown="1">
Expand Down Expand Up @@ -1036,6 +1036,15 @@ for (teenName in collect(teenNames)) {

</div>

<div data-lang="python" markdown="1">

{% highlight python %}
# sqlContext is an existing HiveContext
sqlContext.sql("REFRESH TABLE my_table")
{% endhighlight %}

</div>

<div data-lang="sql" markdown="1">

{% highlight sql %}
Expand All @@ -1054,7 +1063,7 @@ SELECT * FROM parquetTable

</div>

### Partition discovery
### Partition Discovery

Table partitioning is a common optimization approach used in systems like Hive. In a partitioned
table, data are usually stored in different directories, with partitioning column values encoded in
Expand Down Expand Up @@ -1108,7 +1117,7 @@ can be configured by `spark.sql.sources.partitionColumnTypeInference.enabled`, w
`true`. When type inference is disabled, string type will be used for the partitioning columns.


### Schema merging
### Schema Merging

Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with
a simple schema, and gradually add more columns to the schema as needed. In this way, users may end
Expand Down Expand Up @@ -1208,6 +1217,79 @@ printSchema(df3)

</div>

### Hive metastore Parquet table conversion

When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own
Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the
`spark.sql.hive.convertMetastoreParquet` configuration, and is turned on by default.

#### Hive/Parquet Schema Reconciliation

There are two key differences between Hive and Parquet from the perspective of table schema
processing.

1. Hive is case insensitive, while Parquet is not
1. Hive considers all columns nullable, while nullability in Parquet is significant

Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a
Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are:

1. Fields that have the same name in both schema must have the same data type regardless of
nullability. The reconciled field should have the data type of the Parquet side, so that
nullability is respected.

1. The reconciled schema contains exactly those fields defined in Hive metastore schema.

- Any fields that only appear in the Parquet schema are dropped in the reconciled schema.
- Any fileds that only appear in the Hive metastore schema are added as nullable field in the
reconciled schema.

#### Metadata Refreshing

Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table
conversion is enabled, metadata of those converted tables are also cached. If these tables are
updated by Hive or other external tools, you need to refresh them manually to ensure consistent
metadata.

<div class="codetabs">

<div data-lang="scala" markdown="1">

{% highlight scala %}
// sqlContext is an existing HiveContext
sqlContext.refreshTable("my_table")
{% endhighlight %}

</div>

<div data-lang="java" markdown="1">

{% highlight java %}
// sqlContext is an existing HiveContext
sqlContext.refreshTable("my_table")
{% endhighlight %}

</div>

<div data-lang="python" markdown="1">

{% highlight python %}
# sqlContext is an existing HiveContext
sqlContext.refreshTable("my_table")
{% endhighlight %}

</div>

<div data-lang="sql" markdown="1">

{% highlight sql %}
REFRESH TABLE my_table;
{% endhighlight %}

</div>

</div>

### Configuration

Configuration of Parquet can be done using the `setConf` method on `SQLContext` or by running
Expand Down Expand Up @@ -1266,6 +1348,34 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`
support.
</td>
</tr>
<tr>
<td><code>spark.sql.parquet.output.committer.class</code></td>
<td><code>org.apache.parquet.hadoop.<br />ParquetOutputCommitter</code></td>
<td>
<p>
The output committer class used by Parquet. The specified class needs to be a subclass of
<code>org.apache.hadoop.<br />mapreduce.OutputCommitter</code>. Typically, it's also a
subclass of <code>org.apache.parquet.hadoop.ParquetOutputCommitter</code>.
</p>
<p>
<b>Note:</b>
<ul>
<li>
This option must be set via Hadoop <code>Configuration</code> rather than Spark
<code>SQLConf</code>.
</li>
<li>
This option overrides <code>spark.sql.sources.<br />outputCommitterClass</code>.
</li>
</ul>
</p>
<p>
Spark SQL comes with a builtin
<code>org.apache.spark.sql.<br />parquet.DirectParquetOutputCommitter</code>, which can be more
efficient then the default Parquet output committer when writing data to S3.
</p>
</td>
</tr>
</table>

## JSON Datasets
Expand Down Expand Up @@ -1445,8 +1555,8 @@ This command builds a new assembly jar that includes Hive. Note that this Hive a
on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries
(SerDes) in order to access data stored in Hive.

Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. Please note when running
the query on a YARN cluster (`yarn-cluster` mode), the `datanucleus` jars under the `lib_managed/jars` directory
Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. Please note when running
the query on a YARN cluster (`yarn-cluster` mode), the `datanucleus` jars under the `lib_managed/jars` directory
and `hive-site.xml` under `conf/` directory need to be available on the driver and all executors launched by the
YARN cluster. The convenient way to do this is adding them through the `--jars` option and `--file` option of the
`spark-submit` command.
Expand Down Expand Up @@ -1794,7 +1904,7 @@ that these options will be deprecated in future release as more optimizations ar
Configures the number of partitions to use when shuffling data for joins or aggregations.
</td>
</tr>
<tr>
<tr>
<td><code>spark.sql.planner.externalSort</code></td>
<td>false</td>
<td>
Expand Down Expand Up @@ -1889,7 +1999,7 @@ options.
#### DataFrame data reader/writer interface

Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`)
and writing data out (`DataFrame.write`),
and writing data out (`DataFrame.write`),
and deprecated the old APIs (e.g. `SQLContext.parquetFile`, `SQLContext.jsonFile`).

See the API docs for `SQLContext.read` (
Expand Down
56 changes: 35 additions & 21 deletions ec2/spark_ec2.py
Original file line number Diff line number Diff line change
Expand Up @@ -306,6 +306,13 @@ def parse_args():
"--private-ips", action="store_true", default=False,
help="Use private IPs for instances rather than public if VPC/subnet " +
"requires that.")
parser.add_option(
"--instance-initiated-shutdown-behavior", default="stop",
choices=["stop", "terminate"],
help="Whether instances should terminate when shut down or just stop")
parser.add_option(
"--instance-profile-name", default=None,
help="IAM profile name to launch instances under")

(opts, args) = parser.parse_args()
if len(args) != 2:
Expand Down Expand Up @@ -602,7 +609,8 @@ def launch_cluster(conn, opts, cluster_name):
block_device_map=block_map,
subnet_id=opts.subnet_id,
placement_group=opts.placement_group,
user_data=user_data_content)
user_data=user_data_content,
instance_profile_name=opts.instance_profile_name)
my_req_ids += [req.id for req in slave_reqs]
i += 1

Expand Down Expand Up @@ -647,16 +655,19 @@ def launch_cluster(conn, opts, cluster_name):
for zone in zones:
num_slaves_this_zone = get_partition(opts.slaves, num_zones, i)
if num_slaves_this_zone > 0:
slave_res = image.run(key_name=opts.key_pair,
security_group_ids=[slave_group.id] + additional_group_ids,
instance_type=opts.instance_type,
placement=zone,
min_count=num_slaves_this_zone,
max_count=num_slaves_this_zone,
block_device_map=block_map,
subnet_id=opts.subnet_id,
placement_group=opts.placement_group,
user_data=user_data_content)
slave_res = image.run(
key_name=opts.key_pair,
security_group_ids=[slave_group.id] + additional_group_ids,
instance_type=opts.instance_type,
placement=zone,
min_count=num_slaves_this_zone,
max_count=num_slaves_this_zone,
block_device_map=block_map,
subnet_id=opts.subnet_id,
placement_group=opts.placement_group,
user_data=user_data_content,
instance_initiated_shutdown_behavior=opts.instance_initiated_shutdown_behavior,
instance_profile_name=opts.instance_profile_name)
slave_nodes += slave_res.instances
print("Launched {s} slave{plural_s} in {z}, regid = {r}".format(
s=num_slaves_this_zone,
Expand All @@ -678,16 +689,19 @@ def launch_cluster(conn, opts, cluster_name):
master_type = opts.instance_type
if opts.zone == 'all':
opts.zone = random.choice(conn.get_all_zones()).name
master_res = image.run(key_name=opts.key_pair,
security_group_ids=[master_group.id] + additional_group_ids,
instance_type=master_type,
placement=opts.zone,
min_count=1,
max_count=1,
block_device_map=block_map,
subnet_id=opts.subnet_id,
placement_group=opts.placement_group,
user_data=user_data_content)
master_res = image.run(
key_name=opts.key_pair,
security_group_ids=[master_group.id] + additional_group_ids,
instance_type=master_type,
placement=opts.zone,
min_count=1,
max_count=1,
block_device_map=block_map,
subnet_id=opts.subnet_id,
placement_group=opts.placement_group,
user_data=user_data_content,
instance_initiated_shutdown_behavior=opts.instance_initiated_shutdown_behavior,
instance_profile_name=opts.instance_profile_name)

master_nodes = master_res.instances
print("Launched master in %s, regid = %s" % (zone, master_res.id))
Expand Down
3 changes: 1 addition & 2 deletions project/MimaBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,7 @@ object MimaBuild {

def mimaSettings(sparkHome: File, projectRef: ProjectRef) = {
val organization = "org.apache.spark"
// TODO: Change this once Spark 1.4.0 is released
val previousSparkVersion = "1.4.0-rc4"
val previousSparkVersion = "1.4.0"
val fullId = "spark-" + projectRef.project + "_2.10"
mimaDefaultSettings ++
Seq(previousArtifact := Some(organization % fullId % previousSparkVersion),
Expand Down
3 changes: 1 addition & 2 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -166,9 +166,8 @@ object SparkBuild extends PomBuild {
/* Enable tests settings for all projects except examples, assembly and tools */
(allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings))

// TODO: remove launcher from this list after 1.4.0
allProjects.filterNot(x => Seq(spark, hive, hiveThriftServer, catalyst, repl,
networkCommon, networkShuffle, networkYarn, launcher, unsafe).contains(x)).foreach {
networkCommon, networkShuffle, networkYarn, unsafe).contains(x)).foreach {
x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)
}

Expand Down
5 changes: 5 additions & 0 deletions python/pyspark/sql/column.py
Original file line number Diff line number Diff line change
Expand Up @@ -396,6 +396,11 @@ def over(self, window):
jc = self._jc.over(window._jspec)
return Column(jc)

def __nonzero__(self):
raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
"'~' for 'not' when building DataFrame boolean expressions.")
__bool__ = __nonzero__

def __repr__(self):
return 'Column<%s>' % self._jc.toString().encode('utf8')

Expand Down
10 changes: 9 additions & 1 deletion python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,14 @@ def test_explode(self):
self.assertEqual(result[0][0], "a")
self.assertEqual(result[0][1], "b")

def test_and_in_expression(self):
self.assertEqual(4, self.df.filter((self.df.key <= 10) & (self.df.value <= "2")).count())
self.assertRaises(ValueError, lambda: (self.df.key <= 10) and (self.df.value <= "2"))
self.assertEqual(14, self.df.filter((self.df.key <= 3) | (self.df.value < "2")).count())
self.assertRaises(ValueError, lambda: self.df.key <= 3 or self.df.value < "2")
self.assertEqual(99, self.df.filter(~(self.df.key == 1)).count())
self.assertRaises(ValueError, lambda: not self.df.key == 1)

def test_udf_with_callable(self):
d = [Row(number=i, squared=i**2) for i in range(10)]
rdd = self.sc.parallelize(d)
Expand Down Expand Up @@ -408,7 +416,7 @@ def test_column_operators(self):
self.assertTrue(isinstance((- ci - 1 - 2) % 3 * 2.5 / 3.5, Column))
rcc = (1 + ci), (1 - ci), (1 * ci), (1 / ci), (1 % ci)
self.assertTrue(all(isinstance(c, Column) for c in rcc))
cb = [ci == 5, ci != 0, ci > 3, ci < 4, ci >= 0, ci <= 7, ci and cs, ci or cs]
cb = [ci == 5, ci != 0, ci > 3, ci < 4, ci >= 0, ci <= 7]
self.assertTrue(all(isinstance(c, Column) for c in cb))
cbool = (ci & ci), (ci | ci), (~ci)
self.assertTrue(all(isinstance(c, Column) for c in cbool))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,8 +175,10 @@ class CodeGenContext {
* Generate code for compare expression in Java
*/
def genComp(dataType: DataType, c1: String, c2: String): String = dataType match {
// java boolean doesn't support > or < operator
case BooleanType => s"($c1 == $c2 ? 0 : ($c1 ? 1 : -1))"
// use c1 - c2 may overflow
case dt: DataType if isPrimitiveType(dt) => s"(int)($c1 > $c2 ? 1 : $c1 < $c2 ? -1 : 0)"
case dt: DataType if isPrimitiveType(dt) => s"($c1 > $c2 ? 1 : $c1 < $c2 ? -1 : 0)"
case BinaryType => s"org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary($c1, $c2)"
case other => s"$c1.compare($c2)"
}
Expand Down
Loading

0 comments on commit 5484293

Please sign in to comment.