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
396 changes: 358 additions & 38 deletions backends-clickhouse/pom.xml

Large diffs are not rendered by default.

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* 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.gluten.execution.iceberg;

import java.util.Map;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.exceptions.AlreadyExistsException;
import org.apache.iceberg.hive.HiveCatalog;
import org.apache.iceberg.hive.TestHiveMetastore;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkCatalogConfig;
import org.apache.iceberg.spark.source.TestPositionDeletesTable;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.runners.Parameterized;

public class TestPositionDeletesTableGluten extends TestPositionDeletesTable {
private static final Map<String, String> CATALOG_PROPS =
ImmutableMap.of("type", "hive", "default-namespace", "default", "cache-enabled", "false");
private static ClickHouseIcebergHiveTableSupport hiveTableSupport;

@BeforeClass
public static void startMetastoreAndSpark() {
metastore = new TestHiveMetastore();
metastore.start();
hiveConf = metastore.hiveConf();
hiveTableSupport = new ClickHouseIcebergHiveTableSupport();
hiveTableSupport.initSparkConf(
hiveConf.get("hive.metastore.uris"), SparkCatalogConfig.HIVE.catalogName(), null);
hiveTableSupport.initializeSession();
spark = hiveTableSupport.spark();
sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
catalog =
(HiveCatalog)
CatalogUtil.loadCatalog(
HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);

try {
catalog.createNamespace(Namespace.of(new String[] {"default"}));
} catch (AlreadyExistsException ignore) {
}
}

@AfterClass
public static void stopMetastoreAndSpark() throws Exception {
catalog = null;
if (metastore != null) {
metastore.stop();
metastore = null;
}
hiveTableSupport.clean();
}

@Parameterized.Parameters(
name =
"formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}, fileFormat = {4}")
public static Object[][] parameters() {
// ignore ORC and AVRO, ch backend only support PARQUET
return new Object[][] {
{
SparkCatalogConfig.HIVE.catalogName(),
SparkCatalogConfig.HIVE.implementation(),
CATALOG_PROPS,
FileFormat.PARQUET
}
};
}

public TestPositionDeletesTableGluten(
String catalogName, String implementation, Map<String, String> config, FileFormat format) {
super(catalogName, implementation, config, format);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
/*
* 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.gluten.execution.iceberg

import com.google.common.base.Strings
import org.apache.spark.SparkConf
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig

class ClickHouseIcebergHiveTableSupport {

private val sparkConf: SparkConf = new SparkConf()

private var _hiveSpark: SparkSession = _

def spark: SparkSession = _hiveSpark

def initSparkConf(url: String, catalog: String, path: String): SparkConf = {
import org.apache.gluten.backendsapi.clickhouse.CHConfig._

sparkConf
.set("spark.plugins", "org.apache.gluten.GlutenPlugin")
.set("spark.memory.offHeap.enabled", "true")
.set("spark.memory.offHeap.size", "536870912")
.set("spark.sql.catalogImplementation", "hive")
.set("spark.sql.adaptive.enabled", "true")
.set("spark.sql.files.maxPartitionBytes", "1g")
.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer")
.set("spark.sql.shuffle.partitions", "5")
.set("spark.sql.adaptive.enabled", "false")
.set("spark.sql.files.minPartitionNum", "1")
.set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1")
.set("spark.gluten.sql.columnar.iterator", "true")
.set("spark.gluten.sql.columnar.hashagg.enablefinal", "true")
.set("spark.gluten.sql.enable.native.validation", "false")
.set("spark.gluten.sql.parquet.maxmin.index", "true")
.set("spark.hive.exec.dynamic.partition.mode", "nonstrict")
.set("spark.gluten.supported.hive.udfs", "my_add")
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
.set("spark.sql.adaptive.enabled", "true")
.set("spark.sql.shuffle.partitions", "2")
.set("spark.memory.offHeap.size", "2g")
.set("spark.unsafe.exceptionOnMemoryLeak", "true")
.set("spark.sql.autoBroadcastJoinThreshold", "-1")
.setCHConfig("use_local_format", true)
.set("spark.sql.extensions",
"org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
.set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog")
.set("spark.sql.catalog.spark_catalog.type", "hive")
.setMaster("local[*]")
if (!Strings.isNullOrEmpty(url)) {
sparkConf.set("spark.hadoop.hive.metastore.uris", url)
}
if (!Strings.isNullOrEmpty(catalog)) {
sparkConf.set("spark.sql.catalog." + catalog, "org.apache.iceberg.spark.SparkCatalog")
.set("spark.sql.catalog." + catalog + ".type", "hive")
}
if (!Strings.isNullOrEmpty(path)) {
sparkConf.set("spark.sql.warehouse.dir", path)
}
sparkConf
}

def initializeSession(): Unit = {
if (_hiveSpark == null) {
_hiveSpark =
SparkSession
.builder()
.config(sparkConf)
.enableHiveSupport()
.getOrCreate()
}
}

def clean(): Unit = {
try {
if (_hiveSpark != null) {
_hiveSpark.stop()
_hiveSpark = null
}
} finally {
SparkSession.clearActiveSession()
SparkSession.clearDefaultSession()
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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.gluten.execution.iceberg;

import java.util.Map;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.ParameterizedTestExtension;
import org.apache.iceberg.Parameters;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.exceptions.AlreadyExistsException;
import org.apache.iceberg.hive.HiveCatalog;
import org.apache.iceberg.hive.TestHiveMetastore;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkCatalogConfig;
import org.apache.iceberg.spark.source.TestPositionDeletesTable;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.extension.ExtendWith;

@ExtendWith({ParameterizedTestExtension.class})
public class TestPositionDeletesTableGluten extends TestPositionDeletesTable {
private static final Map<String, String> CATALOG_PROPS =
ImmutableMap.of("type", "hive", "default-namespace", "default", "cache-enabled", "false");
private static ClickHouseIcebergHiveTableSupport hiveTableSupport;

@BeforeAll
public static void startMetastoreAndSpark() {
metastore = new TestHiveMetastore();
metastore.start();
hiveConf = metastore.hiveConf();
hiveTableSupport = new ClickHouseIcebergHiveTableSupport();
hiveTableSupport.initSparkConf(
hiveConf.get("hive.metastore.uris"), SparkCatalogConfig.HIVE.catalogName(), null);
hiveTableSupport.initializeSession();
spark = hiveTableSupport.spark();
sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
catalog =
(HiveCatalog)
CatalogUtil.loadCatalog(
HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);

try {
catalog.createNamespace(Namespace.of(new String[] {"default"}));
} catch (AlreadyExistsException ignore) {
}
}

@AfterAll
public static void stopMetastoreAndSpark() throws Exception {
catalog = null;
if (metastore != null) {
metastore.stop();
metastore = null;
}
hiveTableSupport.clean();
}

public TestPositionDeletesTableGluten() {}

@Parameters(name = "catalogName = {1}, implementation = {2}, config = {3}, fileFormat = {4}")
public static Object[][] parameters() {
// ignore ORC and AVRO, ch backend only support PARQUET
return new Object[][] {
{
SparkCatalogConfig.HIVE.catalogName(),
SparkCatalogConfig.HIVE.implementation(),
CATALOG_PROPS,
FileFormat.PARQUET
}
};
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
/*
* 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.gluten.execution.iceberg

import com.google.common.base.Strings
import org.apache.spark.SparkConf
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig

class ClickHouseIcebergHiveTableSupport {

private val sparkConf: SparkConf = new SparkConf()

private var _hiveSpark: SparkSession = _

def spark: SparkSession = _hiveSpark

def initSparkConf(url: String, catalog: String, path: String): SparkConf = {
import org.apache.gluten.backendsapi.clickhouse.CHConfig._

sparkConf
.set("spark.plugins", "org.apache.gluten.GlutenPlugin")
.set("spark.memory.offHeap.enabled", "true")
.set("spark.memory.offHeap.size", "536870912")
.set("spark.sql.catalogImplementation", "hive")
.set("spark.sql.adaptive.enabled", "true")
.set("spark.sql.files.maxPartitionBytes", "1g")
.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer")
.set("spark.sql.shuffle.partitions", "5")
.set("spark.sql.adaptive.enabled", "false")
.set("spark.sql.files.minPartitionNum", "1")
.set(ClickHouseConfig.CLICKHOUSE_WORKER_ID, "1")
.set("spark.gluten.sql.columnar.iterator", "true")
.set("spark.gluten.sql.columnar.hashagg.enablefinal", "true")
.set("spark.gluten.sql.enable.native.validation", "false")
.set("spark.gluten.sql.parquet.maxmin.index", "true")
.set("spark.hive.exec.dynamic.partition.mode", "nonstrict")
.set("spark.gluten.supported.hive.udfs", "my_add")
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
.set("spark.sql.adaptive.enabled", "true")
.set("spark.sql.shuffle.partitions", "2")
.set("spark.memory.offHeap.size", "2g")
.set("spark.unsafe.exceptionOnMemoryLeak", "true")
.set("spark.sql.autoBroadcastJoinThreshold", "-1")
.setCHConfig("use_local_format", true)
.set("spark.sql.extensions",
"org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
.set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog")
.set("spark.sql.catalog.spark_catalog.type", "hive")
.setMaster("local[*]")
if (!Strings.isNullOrEmpty(url)) {
sparkConf.set("spark.hadoop.hive.metastore.uris", url)
}
if (!Strings.isNullOrEmpty(catalog)) {
sparkConf.set("spark.sql.catalog." + catalog, "org.apache.iceberg.spark.SparkCatalog")
.set("spark.sql.catalog." + catalog + ".type", "hive")
}
if (!Strings.isNullOrEmpty(path)) {
sparkConf.set("spark.sql.warehouse.dir", path)
}
sparkConf
}

def initializeSession(): Unit = {
if (_hiveSpark == null) {
_hiveSpark =
SparkSession
.builder()
.config(sparkConf)
.enableHiveSupport()
.getOrCreate()
}
}

def clean(): Unit = {
try {
if (_hiveSpark != null) {
_hiveSpark.stop()
_hiveSpark = null
}
} finally {
SparkSession.clearActiveSession()
SparkSession.clearDefaultSession()
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,9 @@ abstract class GlutenClickHouseCacheBaseTestSuite
override protected val queriesResults: String = rootPath + "queries-output"

// Abstract methods to be implemented by subclasses
protected def cleanupCache(): Unit = cacheHelper.deleteCache(spark, tablesPath)
protected def cleanupCache(): Unit =
cacheHelper.deleteCache(spark, s"$tablesPath/lineitem", s"$tablesPath/$SPARK_DIR_NAME")

protected def copyDataIfNeeded(): Unit

// Initialize the cache helper - accessible to subclasses
Expand Down
Loading