mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Better
This commit is contained in:
parent
3ac7f999bc
commit
b9811573be
@ -332,6 +332,7 @@ class ClickHouseCluster:
|
||||
custom_dockerd_host=None,
|
||||
zookeeper_keyfile=None,
|
||||
zookeeper_certfile=None,
|
||||
with_spark=False,
|
||||
):
|
||||
for param in list(os.environ.keys()):
|
||||
logging.debug("ENV %40s %s" % (param, os.environ[param]))
|
||||
@ -448,7 +449,6 @@ class ClickHouseCluster:
|
||||
self.minio_redirect_ip = None
|
||||
self.minio_redirect_port = 8080
|
||||
|
||||
self.with_spark = False
|
||||
self.spark_session = None
|
||||
|
||||
self.with_azurite = False
|
||||
@ -621,6 +621,19 @@ class ClickHouseCluster:
|
||||
shutil.rmtree(self.instances_dir, ignore_errors=True)
|
||||
logging.debug(f"Removed :{self.instances_dir}")
|
||||
|
||||
if with_spark:
|
||||
# if you change packages, don't forget to update them in docker/test/integration/runner/dockerd-entrypoint.sh
|
||||
(
|
||||
pyspark.sql.SparkSession.builder.appName("spark_test")
|
||||
.config(
|
||||
"spark.jars.packages",
|
||||
"org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0",
|
||||
)
|
||||
.master("local")
|
||||
.getOrCreate()
|
||||
.stop()
|
||||
)
|
||||
|
||||
def print_all_docker_pieces(self):
|
||||
res_networks = subprocess.check_output(
|
||||
f"docker network ls --filter name='{self.project_name}*'",
|
||||
@ -1440,7 +1453,6 @@ class ClickHouseCluster:
|
||||
with_jdbc_bridge=False,
|
||||
with_hive=False,
|
||||
with_coredns=False,
|
||||
with_spark=False,
|
||||
hostname=None,
|
||||
env_variables=None,
|
||||
image="clickhouse/integration-test",
|
||||
@ -1766,18 +1778,6 @@ class ClickHouseCluster:
|
||||
self.setup_hive(instance, env_variables, docker_compose_yml_dir)
|
||||
)
|
||||
|
||||
if with_spark:
|
||||
spark_session = (
|
||||
pyspark.sql.SparkSession.builder.appName("spark_test")
|
||||
.config(
|
||||
"spark.jars.packages",
|
||||
"org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0",
|
||||
)
|
||||
.master("local")
|
||||
.getOrCreate()
|
||||
.stop()
|
||||
)
|
||||
|
||||
logging.debug(
|
||||
"Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format(
|
||||
self.name,
|
||||
|
@ -65,19 +65,5 @@
|
||||
"test_server_reload/test.py::test_remove_http_port",
|
||||
"test_server_reload/test.py::test_remove_mysql_port",
|
||||
"test_server_reload/test.py::test_remove_postgresql_port",
|
||||
"test_server_reload/test.py::test_remove_tcp_port",
|
||||
|
||||
"test_storage_hudi/test.py::test_single_hudi_file",
|
||||
"test_storage_hudi/test.py::test_multiple_hudi_files",
|
||||
"test_storage_hudi/test.py::test_types",
|
||||
"test_storage_delta/test.py::test_single_log_file",
|
||||
"test_storage_delta/test.py::test_partition_by",
|
||||
"test_storage_delta/test.py::test_checkpoint",
|
||||
"test_storage_delta/test.py::test_multiple_log_files",
|
||||
"test_storage_delta/test.py::test_metadata",
|
||||
"test_storage_delta/test.py::test_types",
|
||||
"test_storage_iceberg/test.py::test_single_iceberg_file",
|
||||
"test_storage_iceberg/test.py::test_partition_by",
|
||||
"test_storage_iceberg/test.py::test_multiple_iceberg_files",
|
||||
"test_storage_iceberg/test.py::test_types",
|
||||
"test_server_reload/test.py::test_remove_tcp_port"
|
||||
]
|
||||
|
@ -29,7 +29,7 @@ from pyspark.sql.window import Window
|
||||
|
||||
from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__), with_spark=True)
|
||||
|
||||
|
||||
def get_spark():
|
||||
@ -54,7 +54,6 @@ def started_cluster():
|
||||
"node1",
|
||||
main_configs=["configs/config.d/named_collections.xml"],
|
||||
with_minio=True,
|
||||
with_spark=True,
|
||||
)
|
||||
|
||||
logging.info("Starting cluster...")
|
||||
@ -62,9 +61,6 @@ def started_cluster():
|
||||
|
||||
prepare_s3_bucket(cluster)
|
||||
|
||||
pyspark.sql.SparkSession.builder.appName("spark_test").master(
|
||||
"local"
|
||||
).getOrCreate().stop()
|
||||
cluster.spark_session = get_spark()
|
||||
|
||||
yield cluster
|
||||
|
@ -25,7 +25,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number
|
||||
from pyspark.sql.window import Window
|
||||
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__), with_spark=True)
|
||||
|
||||
|
||||
def get_spark():
|
||||
@ -52,7 +52,6 @@ def started_cluster():
|
||||
"node1",
|
||||
main_configs=["configs/config.d/named_collections.xml"],
|
||||
with_minio=True,
|
||||
with_spark=True,
|
||||
)
|
||||
|
||||
logging.info("Starting cluster...")
|
||||
@ -61,9 +60,6 @@ def started_cluster():
|
||||
prepare_s3_bucket(cluster)
|
||||
logging.info("S3 bucket created")
|
||||
|
||||
pyspark.sql.SparkSession.builder.appName("spark_test").master(
|
||||
"local"
|
||||
).getOrCreate().stop()
|
||||
cluster.spark_session = get_spark()
|
||||
|
||||
yield cluster
|
||||
|
@ -49,12 +49,11 @@ def get_spark():
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster = ClickHouseCluster(__file__, with_spark=True)
|
||||
cluster.add_instance(
|
||||
"node1",
|
||||
main_configs=["configs/config.d/named_collections.xml"],
|
||||
with_minio=True,
|
||||
with_spark=True,
|
||||
)
|
||||
|
||||
logging.info("Starting cluster...")
|
||||
@ -63,9 +62,6 @@ def started_cluster():
|
||||
prepare_s3_bucket(cluster)
|
||||
logging.info("S3 bucket created")
|
||||
|
||||
pyspark.sql.SparkSession.builder.appName("spark_test").master(
|
||||
"local"
|
||||
).getOrCreate().stop()
|
||||
cluster.spark_session = get_spark()
|
||||
|
||||
yield cluster
|
||||
|
Loading…
Reference in New Issue
Block a user