Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,12 @@
import org.apache.arrow.vector.types.pojo.Field;

/** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */
public class CometSchemaImporter {
public abstract class AbstractCometSchemaImporter {
private final BufferAllocator allocator;
private final SchemaImporter importer;
private final CDataDictionaryProvider provider = new CDataDictionaryProvider();

public CometSchemaImporter(BufferAllocator allocator) {
public AbstractCometSchemaImporter(BufferAllocator allocator) {
this.allocator = allocator;
this.importer = new SchemaImporter(allocator);
}
Expand Down
30 changes: 30 additions & 0 deletions common/src/main/java/org/apache/comet/CometSchemaImporter.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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.
*/

package org.apache.comet;

import org.apache.arrow.c.*;
import org.apache.arrow.memory.BufferAllocator;

/** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */
public class CometSchemaImporter extends AbstractCometSchemaImporter {
public CometSchemaImporter(BufferAllocator allocator) {
super(allocator);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.arrow.c.CometSchemaImporter;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.commons.lang3.tuple.Pair;
Expand Down Expand Up @@ -65,6 +64,7 @@
import org.apache.spark.util.AccumulatorV2;

import org.apache.comet.CometConf;
import org.apache.comet.CometSchemaImporter;
import org.apache.comet.shims.ShimBatchReader;
import org.apache.comet.shims.ShimFileFormat;
import org.apache.comet.vector.CometVector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@

import org.apache.arrow.c.ArrowArray;
import org.apache.arrow.c.ArrowSchema;
import org.apache.arrow.c.CometSchemaImporter;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.FieldVector;
Expand All @@ -44,6 +43,7 @@
import org.apache.spark.sql.types.DataType;

import org.apache.comet.CometConf;
import org.apache.comet.CometSchemaImporter;
import org.apache.comet.vector.CometDecodedVector;
import org.apache.comet.vector.CometDictionary;
import org.apache.comet.vector.CometDictionaryVector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,11 @@

import java.io.IOException;

import org.apache.arrow.c.CometSchemaImporter;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.page.PageReader;
import org.apache.spark.sql.types.DataType;

import org.apache.comet.CometSchemaImporter;
import org.apache.comet.vector.CometLazyVector;
import org.apache.comet.vector.CometVector;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.arrow.c.CometSchemaImporter;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.ipc.WriteChannel;
Expand Down Expand Up @@ -71,6 +70,7 @@
import org.apache.spark.util.AccumulatorV2;

import org.apache.comet.CometConf;
import org.apache.comet.CometSchemaImporter;
import org.apache.comet.shims.ShimBatchReader;
import org.apache.comet.shims.ShimFileFormat;
import org.apache.comet.vector.CometVector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,13 +24,13 @@

import org.apache.arrow.c.ArrowArray;
import org.apache.arrow.c.ArrowSchema;
import org.apache.arrow.c.CometSchemaImporter;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.schema.Type;
import org.apache.spark.sql.types.DataType;

import org.apache.comet.CometSchemaImporter;
import org.apache.comet.vector.*;

// TODO: extend ColumnReader instead of AbstractColumnReader to reduce code duplication
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.apache.comet.parquet;

/** This is implemented in Apache Iceberg */
public interface SupportsComet {
boolean isCometEnabled();
}
5 changes: 4 additions & 1 deletion common/src/main/java/org/apache/comet/parquet/Utils.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,16 @@

package org.apache.comet.parquet;

import org.apache.arrow.c.CometSchemaImporter;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.schema.LogicalTypeAnnotation;
import org.apache.parquet.schema.PrimitiveType;
import org.apache.spark.sql.types.*;

import org.apache.comet.CometSchemaImporter;

public class Utils {

/** This method is called from Apache Iceberg. */
public static ColumnReader getColumnReader(
DataType type,
ColumnDescriptor descriptor,
Expand Down
21 changes: 14 additions & 7 deletions docs/source/user-guide/datasources.md
Original file line number Diff line number Diff line change
Expand Up @@ -19,29 +19,36 @@

# Supported Spark Data Sources

## Parquet
## File Formats

### Parquet

When `spark.comet.scan.enabled` is enabled, Parquet scans will be performed natively by Comet if all data types
in the schema are supported. When this option is not enabled, the scan will fall back to Spark. In this case,
enabling `spark.comet.convert.parquet.enabled` will immediately convert the data into Arrow format, allowing native
execution to happen after that, but the process may not be efficient.

## CSV
### CSV

Comet does not provide native CSV scan, but when `spark.comet.convert.csv.enabled` is enabled, data is immediately
converted into Arrow format, allowing native execution to happen after that.

## JSON
### JSON

Comet does not provide native JSON scan, but when `spark.comet.convert.json.enabled` is enabled, data is immediately
converted into Arrow format, allowing native execution to happen after that.

# Supported Storages
## Data Catalogs

### Apache Iceberg

See the dedicated [Comet and Iceberg Guide](iceberg.md).

## Supported Storages

## Local
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(nit) Should we keep ###Local?

In progress
Comet supports most standard storage systems, such as local file system and object storage.

## HDFS
### HDFS

Apache DataFusion Comet native reader seamlessly scans files from remote HDFS for [supported formats](#supported-spark-data-sources)

Expand Down
147 changes: 147 additions & 0 deletions docs/source/user-guide/iceberg.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
<!---
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.
-->

# Accelerating Apache Iceberg Parquet Scans using Comet (Experimental)

**Note: Iceberg integration is a work-in-progress. It is currently necessary to build Iceberg from
source rather than using available artifacts in Maven**

## Build Comet

Run a Maven install so that we can compile Iceberg against latest Comet:

```shell
mvn install -DskipTests
```

Build the release JAR to be used from Spark:

```shell
make release
```

Set `COMET_JAR` env var:

```shell
export COMET_JAR=`pwd`/spark/target/comet-spark-spark3.5_2.12-0.9.0-SNAPSHOT.jar
```

## Build Iceberg

Clone the Iceberg repository.

```shell
git clone git@github.com:apache/iceberg.git
```

It will be necessary to make some small changes to Iceberg:

- Update Gradle files to change Comet version to `0.9.0-SNAPSHOT`.
- Replace `import org.apache.comet.shaded.arrow.c.CometSchemaImporter;` with `import org.apache.comet.CometSchemaImporter;`
- Modify `SparkBatchQueryScan` so that it implements the `SupportsComet` interface
- Stop shading Parquet by commenting out the following lines in the iceberg-spark build:

```
// relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet'
// relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded'
```

Perform a clean build

```shell
./gradlew clean
./gradlew build -x test -x integrationTest
```

## Test

Set `ICEBERG_JAR` environment variable.

```shell
export ICEBERG_JAR=`pwd`/spark/v3.5/spark-runtime/build/libs/iceberg-spark-runtime-3.5_2.12-1.10.0-SNAPSHOT.jar
```

Launch Spark Shell:

```shell
$SPARK_HOME/bin/spark-shell \
--jars $COMET_JAR,$ICEBERG_JAR \
--conf spark.driver.extraClassPath=$COMET_JAR,$ICEBERG_JAR \
--conf spark.executor.extraClassPath=$COMET_JAR,$ICEBERG_JAR \
--conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \
--conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkCatalog \
--conf spark.sql.catalog.spark_catalog.type=hadoop \
--conf spark.sql.catalog.spark_catalog.warehouse=/tmp/warehouse \
--conf spark.plugins=org.apache.spark.CometPlugin \
--conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \
--conf spark.sql.iceberg.parquet.reader-type=COMET \
--conf spark.comet.explainFallback.enabled=true \
--conf spark.memory.offHeap.enabled=true \
--conf spark.memory.offHeap.size=16g
```

Create an Iceberg table. Note that Comet will not accelerate this part.

```
scala> spark.sql(s"CREATE TABLE IF NOT EXISTS t1 (c0 INT, c1 STRING) USING iceberg")
scala> spark.sql(s"INSERT INTO t1 VALUES ${(0 until 10000).map(i => (i, i)).mkString(",")}")
```

Comet should now be able to accelerate reading the table:

```
scala> spark.sql(s"SELECT * from t1").show()
```

This should produce the following output:

```
scala> spark.sql(s"SELECT * from t1").show()
25/04/28 07:29:37 INFO core/src/lib.rs: Comet native library version 0.9.0 initialized
25/04/28 07:29:37 WARN CometSparkSessionExtensions$CometExecRule: Comet cannot execute some parts of this plan natively (set spark.comet.explainFallback.enabled=false to disable this logging):
CollectLimit [COMET: CollectLimit is not supported]
+- Project [COMET: toprettystring is not supported]
+- CometScanWrapper

+---+---+
| c0| c1|
+---+---+
| 0| 0|
| 1| 1|
| 2| 2|
| 3| 3|
| 4| 4|
| 5| 5|
| 6| 6|
| 7| 7|
| 8| 8|
| 9| 9|
| 10| 10|
| 11| 11|
| 12| 12|
| 13| 13|
| 14| 14|
| 15| 15|
| 16| 16|
| 17| 17|
| 18| 18|
| 19| 19|
+---+---+
only showing top 20 rows
```
Original file line number Diff line number Diff line change
Expand Up @@ -230,8 +230,11 @@ class CometSparkSessionExtensions
withInfo(scanExec, fallbackReasons.mkString(", "))
}

case _ =>
withInfo(scanExec, "Comet Scan only supports Parquet and Iceberg Parquet file formats")
case other =>
withInfo(
scanExec,
s"Unsupported scan: ${other.getClass.getName}. " +
"Comet Scan only supports Parquet and Iceberg Parquet file formats")
}
}

Expand Down
Loading