mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into ci_skip_never_fails_integr_test
This commit is contained in:
commit
57f9e0cbb2
@ -1,7 +1,7 @@
|
||||
version: '2.3'
|
||||
services:
|
||||
mysql_client:
|
||||
image: mysql:5.7
|
||||
image: mysql:8.0
|
||||
restart: always
|
||||
environment:
|
||||
MYSQL_ALLOW_EMPTY_PASSWORD: 1
|
||||
|
@ -1,8 +1,10 @@
|
||||
import base64
|
||||
import errno
|
||||
from functools import cache
|
||||
import http.client
|
||||
import logging
|
||||
import os
|
||||
import platform
|
||||
import stat
|
||||
import os.path as p
|
||||
import pprint
|
||||
@ -4743,3 +4745,8 @@ class ClickHouseKiller(object):
|
||||
|
||||
def __exit__(self, exc_type, exc_val, exc_tb):
|
||||
self.clickhouse_node.start_clickhouse()
|
||||
|
||||
|
||||
@cache
|
||||
def is_arm():
|
||||
return any(arch in platform.processor().lower() for arch in ("arm, aarch"))
|
||||
|
@ -1,5 +1,5 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance("node1", main_configs=["configs/config_with_hosts.xml"])
|
||||
@ -16,9 +16,11 @@ node5 = cluster.add_instance(
|
||||
"node5", main_configs=["configs/config_without_allowed_hosts.xml"]
|
||||
)
|
||||
node6 = cluster.add_instance("node6", main_configs=["configs/config_for_remote.xml"])
|
||||
node7 = cluster.add_instance(
|
||||
"node7", main_configs=["configs/config_for_redirect.xml"], with_hdfs=True
|
||||
)
|
||||
|
||||
if not is_arm():
|
||||
node7 = cluster.add_instance(
|
||||
"node7", main_configs=["configs/config_for_redirect.xml"], with_hdfs=True
|
||||
)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
@ -270,6 +272,7 @@ def test_table_function_remote(start_cluster):
|
||||
)
|
||||
|
||||
|
||||
@pytest.mark.skipif(is_arm(), reason="skip for ARM")
|
||||
def test_redirect(start_cluster):
|
||||
hdfs_api = start_cluster.hdfs_api
|
||||
|
||||
@ -284,6 +287,7 @@ def test_redirect(start_cluster):
|
||||
node7.query("DROP TABLE table_test_7_1")
|
||||
|
||||
|
||||
@pytest.mark.skipif(is_arm(), reason="skip for ARM")
|
||||
def test_HDFS(start_cluster):
|
||||
assert "not allowed" in node7.query_and_get_error(
|
||||
"CREATE TABLE table_test_7_2 (word String) ENGINE=HDFS('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'CSV')"
|
||||
@ -293,6 +297,7 @@ def test_HDFS(start_cluster):
|
||||
)
|
||||
|
||||
|
||||
@pytest.mark.skipif(is_arm(), reason="skip for ARM")
|
||||
def test_schema_inference(start_cluster):
|
||||
error = node7.query_and_get_error("desc url('http://test.com`, 'TSVRaw'')")
|
||||
assert error.find("ReadWriteBufferFromHTTPBase") == -1
|
||||
|
@ -1,14 +1,17 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
disk_types = {
|
||||
"default": "Local",
|
||||
"disk_s3": "S3",
|
||||
"disk_hdfs": "HDFS",
|
||||
"disk_encrypted": "S3",
|
||||
}
|
||||
|
||||
# do not test HDFS on ARM
|
||||
if not is_arm():
|
||||
disk_types["disk_hdfs"] = "HDFS"
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster():
|
||||
@ -18,7 +21,7 @@ def cluster():
|
||||
"node",
|
||||
main_configs=["configs/storage.xml"],
|
||||
with_minio=True,
|
||||
with_hdfs=True,
|
||||
with_hdfs=not is_arm(),
|
||||
)
|
||||
cluster.start()
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
from helpers.test_tools import TSV
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
@ -10,6 +10,9 @@ disk_types = {
|
||||
"disk_encrypted": "S3",
|
||||
}
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster():
|
||||
|
@ -8,9 +8,13 @@ from confluent_kafka.avro.cached_schema_registry_client import (
|
||||
CachedSchemaRegistryClient,
|
||||
)
|
||||
from confluent_kafka.avro.serializer.message_serializer import MessageSerializer
|
||||
from helpers.cluster import ClickHouseCluster, ClickHouseInstance
|
||||
from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm
|
||||
from urllib import parse
|
||||
|
||||
# Skip on ARM due to Confluent/Kafka
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
|
@ -3,7 +3,7 @@ import os.path as p
|
||||
import pytest
|
||||
import uuid
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
from helpers.test_tools import TSV
|
||||
from string import Template
|
||||
|
||||
@ -14,6 +14,9 @@ instance = cluster.add_instance(
|
||||
datasource = "self"
|
||||
records = 1000
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
|
@ -2,10 +2,14 @@ import time
|
||||
import logging
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection
|
||||
from kafka.admin import NewTopic
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance(
|
||||
"instance",
|
||||
|
@ -1,11 +1,15 @@
|
||||
#!/usr/bin/env python3
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
import helpers.keeper_utils as keeper_utils
|
||||
from minio.deleteobjects import DeleteObject
|
||||
|
||||
import os
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__))
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node_logs = cluster.add_instance(
|
||||
|
@ -1,5 +1,9 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance1 = cluster.add_instance(
|
||||
|
@ -2,10 +2,13 @@ import logging
|
||||
import sys
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
|
@ -6,6 +6,7 @@ from helpers.cluster import (
|
||||
ClickHouseCluster,
|
||||
ClickHouseInstance,
|
||||
get_docker_compose_path,
|
||||
is_arm,
|
||||
)
|
||||
import logging
|
||||
|
||||
@ -13,6 +14,10 @@ from . import materialized_with_ddl
|
||||
|
||||
DOCKER_COMPOSE_PATH = get_docker_compose_path()
|
||||
|
||||
# skip all test on arm due to no arm support in mysql57
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
mysql_node = None
|
||||
mysql8_node = None
|
||||
|
@ -3,7 +3,7 @@ import time
|
||||
import os
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
from helpers.utility import generate_values
|
||||
from helpers.wait_for_helpers import wait_for_delete_inactive_parts
|
||||
from helpers.wait_for_helpers import wait_for_delete_empty_parts
|
||||
@ -16,6 +16,10 @@ CONFIG_PATH = os.path.join(
|
||||
)
|
||||
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
def create_table(cluster, table_name, additional_settings=None):
|
||||
node = cluster.instances["node"]
|
||||
|
||||
|
@ -12,11 +12,16 @@ from typing import Literal
|
||||
import docker
|
||||
import pymysql.connections
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
|
||||
from helpers.cluster import (
|
||||
ClickHouseCluster,
|
||||
get_docker_compose_path,
|
||||
run_and_check,
|
||||
)
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
DOCKER_COMPOSE_PATH = get_docker_compose_path()
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance(
|
||||
"node",
|
||||
|
@ -1,10 +1,15 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
|
||||
from helpers.network import PartitionManager
|
||||
import threading
|
||||
import time
|
||||
|
||||
# skip all tests in the module on ARM due to HDFS
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance(
|
||||
"node1",
|
||||
|
@ -2,10 +2,13 @@ import os
|
||||
|
||||
import pytest
|
||||
import time
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
from helpers.test_tools import TSV
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance(
|
||||
"node1",
|
||||
|
@ -22,7 +22,7 @@ import kafka.errors
|
||||
import pytest
|
||||
from google.protobuf.internal.encoder import _VarintBytes
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
from helpers.network import PartitionManager
|
||||
from helpers.test_tools import TSV
|
||||
from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection
|
||||
@ -40,6 +40,8 @@ from . import kafka_pb2
|
||||
from . import social_pb2
|
||||
from . import message_with_repeated_pb2
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
# TODO: add test for run-time offset update in CH, if we manually update it on Kafka side.
|
||||
# TODO: add test for SELECT LIMIT is working.
|
||||
|
@ -3,9 +3,12 @@ import pytest
|
||||
|
||||
import os
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, is_arm
|
||||
import subprocess
|
||||
|
||||
if is_arm():
|
||||
pytestmark = pytest.mark.skip
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance(
|
||||
"node1",
|
||||
|
Loading…
Reference in New Issue
Block a user