Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[spark] Support report partitioning to eliminate shuffle exchange #3912

Merged
merged 1 commit into from
Aug 8, 2024
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 @@ -18,6 +18,7 @@

package org.apache.paimon.schema;

import org.apache.paimon.CoreOptions;
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.Path;
import org.apache.paimon.types.DataField;
Expand Down Expand Up @@ -66,6 +67,10 @@ public class TableSchema implements Serializable {

private final List<String> primaryKeys;

private final List<String> bucketKeys;

private final int numBucket;

private final Map<String, String> options;

private final @Nullable String comment;
Expand Down Expand Up @@ -115,8 +120,13 @@ public TableSchema(
// try to trim to validate primary keys
trimmedPrimaryKeys();

// try to validate bucket keys
originalBucketKeys();
// try to validate and initalize the bucket keys
List<String> tmpBucketKeys = originalBucketKeys();
if (tmpBucketKeys.isEmpty()) {
tmpBucketKeys = trimmedPrimaryKeys();
}
bucketKeys = tmpBucketKeys;
numBucket = CoreOptions.fromMap(options).bucket();
}

public int version() {
Expand Down Expand Up @@ -171,11 +181,11 @@ public Map<String, String> options() {
return options;
}

public int numBuckets() {
return numBucket;
}

public List<String> bucketKeys() {
List<String> bucketKeys = originalBucketKeys();
if (bucketKeys.isEmpty()) {
bucketKeys = trimmedPrimaryKeys();
}
return bucketKeys;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -150,11 +150,6 @@ public Optional<Statistics> statistics() {
return Optional.empty();
}

@Override
public BucketMode bucketMode() {
return store().bucketMode();
}

@Override
public Optional<WriteSelector> newWriteSelector() {
switch (bucketMode()) {
Expand Down
65 changes: 65 additions & 0 deletions paimon-core/src/main/java/org/apache/paimon/table/BucketSpec.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.paimon.table;

import java.util.List;

/**
* Bucket spec holds all bucket information, we can do plan optimization during table scan.
*
* <p>If the `bucketMode` is {@link BucketMode#HASH_DYNAMIC}, then `numBucket` is -1;
*
* @since 0.9
*/
public class BucketSpec {

private final BucketMode bucketMode;
private final List<String> bucketKeys;
private final int numBuckets;

public BucketSpec(BucketMode bucketMode, List<String> bucketKeys, int numBuckets) {
this.bucketMode = bucketMode;
this.bucketKeys = bucketKeys;
this.numBuckets = numBuckets;
}

public BucketMode getBucketMode() {
return bucketMode;
}

public List<String> getBucketKeys() {
return bucketKeys;
}

public int getNumBuckets() {
return numBuckets;
}

@Override
public String toString() {
return "BucketSpec{"
+ "bucketMode="
+ bucketMode
+ ", bucketKeys="
+ bucketKeys
+ ", numBuckets="
+ numBuckets
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -113,11 +113,6 @@ public FileStore<?> store() {
return wrapped.store();
}

@Override
public BucketMode bucketMode() {
return wrapped.bucketMode();
}

@Override
public CatalogEnvironment catalogEnvironment() {
return wrapped.catalogEnvironment();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,14 @@ default List<String> primaryKeys() {
return schema().primaryKeys();
}

default BucketSpec bucketSpec() {
return new BucketSpec(bucketMode(), schema().bucketKeys(), schema().numBuckets());
}

default BucketMode bucketMode() {
return store().bucketMode();
}

@Override
default Map<String, String> options() {
return schema().options();
Expand All @@ -73,8 +81,6 @@ default Optional<String> comment() {

FileStore<?> store();

BucketMode bucketMode();

CatalogEnvironment catalogEnvironment();

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.paimon.spark.catalog;

import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.spark.SparkProcedures;
import org.apache.paimon.spark.SparkSource;
import org.apache.paimon.spark.analysis.NoSuchProcedureException;
import org.apache.paimon.spark.procedure.Procedure;
import org.apache.paimon.spark.procedure.ProcedureBuilder;

import org.apache.spark.sql.connector.catalog.Identifier;
import org.apache.spark.sql.connector.catalog.SupportsNamespaces;
import org.apache.spark.sql.connector.catalog.TableCatalog;

/** Spark base catalog. */
public abstract class SparkBaseCatalog
implements TableCatalog, SupportsNamespaces, ProcedureCatalog, WithPaimonCatalog {

protected String catalogName;

@Override
public String name() {
return catalogName;
}

@Override
public Procedure loadProcedure(Identifier identifier) throws NoSuchProcedureException {
if (Catalog.SYSTEM_DATABASE_NAME.equals(identifier.namespace()[0])) {
ProcedureBuilder builder = SparkProcedures.newBuilder(identifier.name());
if (builder != null) {
return builder.withTableCatalog(this).build();
}
}
throw new NoSuchProcedureException(identifier);
}

public boolean usePaimon(String provider) {
return provider == null || SparkSource.NAME().equalsIgnoreCase(provider);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.paimon.spark

import org.apache.paimon.table.source.Split

// never be used
case class PaimonBucketedInputPartition(splits: Seq[Split], bucket: Int)
extends PaimonInputPartition
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.paimon.spark.catalog;

import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.spark.SparkProcedures;
import org.apache.paimon.spark.SparkSource;
import org.apache.paimon.spark.analysis.NoSuchProcedureException;
import org.apache.paimon.spark.procedure.Procedure;
import org.apache.paimon.spark.procedure.ProcedureBuilder;

import org.apache.spark.sql.connector.catalog.Identifier;
import org.apache.spark.sql.connector.catalog.SupportsNamespaces;
import org.apache.spark.sql.connector.catalog.TableCatalog;

/** Spark base catalog. */
public abstract class SparkBaseCatalog
implements TableCatalog, SupportsNamespaces, ProcedureCatalog, WithPaimonCatalog {

protected String catalogName;

@Override
public String name() {
return catalogName;
}

@Override
public Procedure loadProcedure(Identifier identifier) throws NoSuchProcedureException {
if (Catalog.SYSTEM_DATABASE_NAME.equals(identifier.namespace()[0])) {
ProcedureBuilder builder = SparkProcedures.newBuilder(identifier.name());
if (builder != null) {
return builder.withTableCatalog(this).build();
}
}
throw new NoSuchProcedureException(identifier);
}

public boolean usePaimon(String provider) {
return provider == null || SparkSource.NAME().equalsIgnoreCase(provider);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.paimon.spark

import org.apache.paimon.table.source.Split

// never be used
case class PaimonBucketedInputPartition(splits: Seq[Split], bucket: Int)
extends PaimonInputPartition
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
/*
* 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.paimon.spark

import org.apache.paimon.predicate.Predicate
import org.apache.paimon.table.Table

import org.apache.spark.sql.PaimonUtils.fieldReference
import org.apache.spark.sql.connector.expressions.NamedReference
import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
import org.apache.spark.sql.sources.{Filter, In}
import org.apache.spark.sql.types.StructType

import scala.collection.JavaConverters._

case class PaimonScan(
table: Table,
requiredSchema: StructType,
filters: Seq[Predicate],
reservedFilters: Seq[Filter],
pushDownLimit: Option[Int])
extends PaimonBaseScan(table, requiredSchema, filters, reservedFilters, pushDownLimit)
with SupportsRuntimeFiltering {

override def filterAttributes(): Array[NamedReference] = {
val requiredFields = readBuilder.readType().getFieldNames.asScala
table
.partitionKeys()
.asScala
.toArray
.filter(requiredFields.contains)
.map(fieldReference)
}

override def filter(filters: Array[Filter]): Unit = {
val converter = new SparkFilterConverter(table.rowType())
val partitionFilter = filters.flatMap {
case in @ In(attr, _) if table.partitionKeys().contains(attr) =>
Some(converter.convert(in))
case _ => None
}
if (partitionFilter.nonEmpty) {
this.runtimeFilters = filters
readBuilder.withFilter(partitionFilter.head)
// set inputPartitions null to trigger to get the new splits.
inputPartitions = null
}
}

}
Loading
Loading