Skip to content
Closed
Show file tree
Hide file tree
Changes from 8 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
34 changes: 34 additions & 0 deletions exporter/clickhouseexporter/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,10 @@ type Config struct {
MetricsTableName string `mapstructure:"metrics_table_name"`
// TTLDays is The data time-to-live in days, 0 means no ttl.
TTLDays uint `mapstructure:"ttl_days"`
// Used to generate ENGINE value when create table. See `TableEngineString` function for details.
TableEngine TableEngine `mapstructure:"table_engine"`
// If set then is used on database and table creation. For example: CREATE DATABASE IF NOT EXISTS db1 ON CLUSTER `ClusterName`
ClusterName string `mapstructure:"cluster_name"`
}

// QueueSettings is a subset of exporterhelper.QueueSettings.
Expand All @@ -48,7 +52,14 @@ type QueueSettings struct {
QueueSize int `mapstructure:"queue_size"`
}

// Encapsulates ENGINE value when create table. For example, Name = ReplicatedReplacingMergeTree and Params = "'par1', 'par2', par3".
type TableEngine struct {
Name string `mapstructure:"name"`
Params string `mapstructure:"params"`
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
Params string `mapstructure:"params"`
Zoo_path_prefix`mapstructure:"zoo_path_prefix"`
Replica_name: `mapstructure:"replica_name"`
Other_parameters: `mapstructure:"other_parameters"`

zoo_path_prefix is the prefix of /clickhouse/tables/{shard}/table_name, for trace table, it will be /clickhouse/tables/{shard}/otel_traces.

Copy link
Author

Choose a reason for hiding this comment

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

The idea behind TableEngine field is to be able to construct any engine with variable list of params like - AnyEngine(p1, p2, p3, p4, p5), it is not necessary have to be related to replication. Am I missing something?

Copy link
Contributor

Choose a reason for hiding this comment

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

AnyEngine(p1, p2, p3, p4, p5) is a general idea, but we need carefully handle the zoo_path param in Replicated*MergeTree, we expect the table creation query string like:

For traces:

      CREATE TABLE IF NOT EXISTS otel.otel_traces on cluster 'otel' (
      ...
      ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/otel_traces', '{replica}');

For logs:

      CREATE TABLE IF NOT EXISTS otel.otel_logs on cluster 'otel' (
      ...
      ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/otel_logs', '{replica}');

instead of every ReplicatedMergeTree uses the same zoo_path.

Copy link
Author

Choose a reason for hiding this comment

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

If we set Config.TableEngine to TableEngine{Name: ReplicatedMergeTree} then generated query will have cluase ENGINE = ReplicatedMergeTree(). Clickhouse transforms it to ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}'). We can check it with show create table otel.otel_logs. Next we can check that placehoders actually generate unique zookeper_path for each table by executing:

SELECT
    fqdn(),replica_name,
    zookeeper_path,
    replica_path
FROM clusterAllReplicas('my_cluster', system.replicas)
WHERE (table = 'otel_logs') AND (database = 'otel')

Here is output example for otel_traces and otel_logs respectively:

┌─FQDN()────┬─replica_name─┬─zookeeper_path─────────────────────────────────────────────┬─replica_path───────────────────────────────────────────────────────────┐
│ localhost │ 02           │ /clickhouse/tables/052912e1-9697-4d2f-8241-4927e15062d0/01 │ /clickhouse/tables/052912e1-9697-4d2f-8241-4927e15062d0/01/replicas/02 │
└───────────┴──────────────┴────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────┘

┌─FQDN()────┬─replica_name─┬─zookeeper_path─────────────────────────────────────────────┬─replica_path───────────────────────────────────────────────────────────┐
│ localhost │ 02           │ /clickhouse/tables/8e490fbe-9857-4d25-90af-c2fffbed22b8/01 │ /clickhouse/tables/8e490fbe-9857-4d25-90af-c2fffbed22b8/01/replicas/02 │
└───────────┴──────────────┴────────────────────────────────────────────────────────────┴────────────────────────────────────────────────────────────────────────┘

As we can see different path is generated.
On the other hand, if user wish to provide ReplicatedMergeTree params then it is user's responsibility to do it correctly.

Copy link
Contributor

Choose a reason for hiding this comment

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

thank you for your clarify

}

const defaultDatabase = "default"
const defaultTableEngine = "MergeTree"

var (
errConfigNoEndpoint = errors.New("endpoint must be specified")
Expand Down Expand Up @@ -142,3 +153,26 @@ func (cfg *Config) buildDB(database string) (*sql.DB, error) {
return conn, nil

}

// Generate ENGINE string
Copy link
Member

Choose a reason for hiding this comment

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

Comments should begin with the name of the thing being described and end in a period. https://github.com/golang/go/wiki/CodeReviewComments#comment-sentences

// If `TableEngine.Name` and `TableEngine.Params` are set then return 'TableEngine.Name(TableEngine.Params)'. If `TableEngine.Params`is empty then return 'TableEngine.Name()'. Otherwise return 'defaultTableEngine()'.
func (cfg *Config) TableEngineString() (string) {
if cfg.TableEngine.Name == "" {
return fmt.Sprintf("%s()", defaultTableEngine)
}

if cfg.TableEngine.Params == "" {
return fmt.Sprintf("%s()", cfg.TableEngine.Name)
}

return fmt.Sprintf("%s(%s)", cfg.TableEngine.Name, cfg.TableEngine.Params)
}

// Produce `ON CLUSTER ClusterName` if `ClusterName` is not empty. Otherwise return "".
func (cfg *Config) ClusterClause() (string) {
if cfg.ClusterName == "" {
return ""
}

return fmt.Sprintf("ON CLUSTER %s", cfg.ClusterName)
}
45 changes: 45 additions & 0 deletions exporter/clickhouseexporter/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,51 @@ import (
)

const defaultEndpoint = "clickhouse://127.0.0.1:9000"
const replicationEndpoint = "tcp://127.0.0.1:19000"
const replicationEndpoint2 = "tcp://127.0.0.1:19001"
const replicationCluster = "cluster_1S_2R"

func TestTableEngineConfigParsing(t *testing.T) {
t.Parallel()
cm, err := confmaptest.LoadConf(filepath.Join("testdata", "config-table-engine.yml"))
require.NoError(t, err)

tests := []struct {
id component.ID
expected string
} {
{
id: component.NewIDWithName(metadata.Type, "table-engine-empty"),
expected: "MergeTree()",
},
{
id: component.NewIDWithName(metadata.Type, "table-engine-name-only"),
expected: "ReplicatedReplacingMergeTree()",
},
{
id: component.NewIDWithName(metadata.Type, "table-engine-full"),
expected: "ReplicatedReplacingMergeTree('/clickhouse/tables/{shard}/table_name', '{replica}', ver)",
},
{
id: component.NewIDWithName(metadata.Type, "table-engine-invalid"),
expected: "MergeTree()",
},
}

for _, tt := range tests {
t.Run(tt.id.String(), func(t *testing.T) {
factory := NewFactory()
cfg := factory.CreateDefaultConfig()

sub, err := cm.Sub(tt.id.String())
require.NoError(t, err)
require.NoError(t, component.UnmarshalConfig(sub, cfg))

assert.NoError(t, component.ValidateConfig(cfg))
assert.Equal(t, tt.expected, cfg.(*Config).TableEngineString())
})
}
}

func TestLoadConfig(t *testing.T) {
t.Parallel()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
<clickhouse>
<macros>
<shard>01</shard>
<replica>01</replica>
<cluster>cluster_1S_2R</cluster>
</macros>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
<clickhouse>
<logger>
<level>debug</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>3</count>
</logger>
<display_name>cluster_1S_2R node 1</display_name>
<listen_host>0.0.0.0</listen_host>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
<clickhouse>
<remote_servers replace="true">
<cluster_1S_2R>
<secret>mysecretphrase</secret>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>clickhouse-01</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse-02</host>
<port>9000</port>
</replica>
</shard>
</cluster_1S_2R>
</remote_servers>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,9 @@
<clickhouse>
<zookeeper>
<!-- where are the ZK nodes -->
<node>
<host>clickhouse-keeper-01</host>
<port>9181</port>
</node>
</zookeeper>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
<?xml version="1.0"?>
<clickhouse replace="true">
<profiles>
<default>
<max_memory_usage>10000000000</max_memory_usage>
<use_uncompressed_cache>0</use_uncompressed_cache>
<load_balancing>in_order</load_balancing>
<log_queries>1</log_queries>
</default>
</profiles>
<users>
<default>
<access_management>1</access_management>
<profile>default</profile>
<networks>
<ip>::/0</ip>
</networks>
<quota>default</quota>
</default>
<admin>
<access_management>1</access_management>
<password>password</password>
<profile>default</profile>
<networks>
<ip>::/0</ip>
</networks>
<quota>default</quota>
</admin>
</users>
<quotas>
<default>
<interval>
<duration>3600</duration>
<queries>0</queries>
<errors>0</errors>
<result_rows>0</result_rows>
<read_rows>0</read_rows>
<execution_time>0</execution_time>
</interval>
</default>
</quotas>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
<clickhouse>
<macros>
<shard>01</shard>
<replica>02</replica>
<cluster>cluster_1S_2R</cluster>
</macros>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
<clickhouse>
<logger>
<level>debug</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>3</count>
</logger>
<display_name>cluster_1S_2R node 2</display_name>
<listen_host>0.0.0.0</listen_host>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
<clickhouse>
<remote_servers replace="true">
<cluster_1S_2R>
<secret>mysecretphrase</secret>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>clickhouse-01</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse-02</host>
<port>9000</port>
</replica>
</shard>
</cluster_1S_2R>
</remote_servers>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,9 @@
<clickhouse>
<zookeeper>
<!-- where are the ZK nodes -->
<node>
<host>clickhouse-keeper-01</host>
<port>9181</port>
</node>
</zookeeper>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
<?xml version="1.0"?>
<clickhouse replace="true">
<profiles>
<default>
<max_memory_usage>10000000000</max_memory_usage>
<use_uncompressed_cache>0</use_uncompressed_cache>
<load_balancing>in_order</load_balancing>
<log_queries>1</log_queries>
</default>
</profiles>
<users>
<default>
<access_management>1</access_management>
<profile>default</profile>
<networks>
<ip>::/0</ip>
</networks>
<quota>default</quota>
</default>
<admin>
<access_management>1</access_management>
<password>password</password>
<profile>default</profile>
<networks>
<ip>::/0</ip>
</networks>
<quota>default</quota>
</admin>
</users>
<quotas>
<default>
<interval>
<duration>3600</duration>
<queries>0</queries>
<errors>0</errors>
<result_rows>0</result_rows>
<read_rows>0</read_rows>
<execution_time>0</execution_time>
</interval>
</default>
</quotas>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
<clickhouse>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-keeper/clickhouse-keeper.log</log>
<errorlog>/var/log/clickhouse-keeper/clickhouse-keeper.err.log</errorlog>
<size>1000M</size>
<count>3</count>
</logger>
<listen_host>0.0.0.0</listen_host>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>clickhouse-keeper-01</hostname>
<port>9234</port>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
version: '1.0'
services:
clickhouse-keeper-01:
image: "clickhouse/clickhouse-keeper:${CHKVER:-latest-alpine}"
user: "101:101"
container_name: clickhouse-keeper-01
hostname: clickhouse-keeper-01
volumes:
- ${PWD}/clickhouse-keeper-01/etc/clickhouse-keeper/keeper_config.xml:/etc/clickhouse-keeper/keeper_config.xml
ports:
- "127.0.0.1:9181:9181"
clickhouse-01:
image: "clickhouse/clickhouse-server:${CHVER:-latest}"
user: "101:101"
container_name: clickhouse-01
hostname: clickhouse-01
volumes:
- ${PWD}/clickhouse-01/etc/clickhouse-server/config.d:/etc/clickhouse-server/config.d
- ${PWD}/clickhouse-01/etc/clickhouse-server/users.d:/etc/clickhouse-server/users.d
ports:
- "127.0.0.1:18123:8123"
- "127.0.0.1:19000:9000"
depends_on:
- clickhouse-keeper-01
clickhouse-02:
image: "clickhouse/clickhouse-server:${CHVER:-latest}"
user: "101:101"
container_name: clickhouse-02
hostname: clickhouse-02
volumes:
- ${PWD}/clickhouse-02/etc/clickhouse-server/config.d:/etc/clickhouse-server/config.d
- ${PWD}/clickhouse-02/etc/clickhouse-server/users.d:/etc/clickhouse-server/users.d
ports:
- "127.0.0.1:18124:8123"
- "127.0.0.1:19001:9000"
depends_on:
- clickhouse-keeper-01
9 changes: 5 additions & 4 deletions exporter/clickhouseexporter/exporter_logs.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ func attributesToMap(attributes pcommon.Map) map[string]string {
const (
// language=ClickHouse SQL
createLogsTableSQL = `
CREATE TABLE IF NOT EXISTS %s (
CREATE TABLE IF NOT EXISTS %s %s (
Timestamp DateTime64(9) CODEC(Delta, ZSTD(1)),
TraceId String CODEC(ZSTD(1)),
SpanId String CODEC(ZSTD(1)),
Expand All @@ -152,7 +152,7 @@ CREATE TABLE IF NOT EXISTS %s (
INDEX idx_log_attr_key mapKeys(LogAttributes) TYPE bloom_filter(0.01) GRANULARITY 1,
INDEX idx_log_attr_value mapValues(LogAttributes) TYPE bloom_filter(0.01) GRANULARITY 1,
INDEX idx_body Body TYPE tokenbf_v1(32768, 3, 0) GRANULARITY 1
) ENGINE MergeTree()
) ENGINE = %s
%s
PARTITION BY toDate(Timestamp)
ORDER BY (ServiceName, SeverityText, toUnixTimestamp(Timestamp), TraceId)
Expand Down Expand Up @@ -218,7 +218,7 @@ func createDatabase(ctx context.Context, cfg *Config) error {
defer func() {
_ = db.Close()
}()
query := fmt.Sprintf("CREATE DATABASE IF NOT EXISTS %s", cfg.Database)
query := fmt.Sprintf("CREATE DATABASE IF NOT EXISTS %s %s", cfg.Database, cfg.ClusterClause())
_, err = db.ExecContext(ctx, query)
if err != nil {
return fmt.Errorf("create database:%w", err)
Expand All @@ -238,7 +238,8 @@ func renderCreateLogsTableSQL(cfg *Config) string {
if cfg.TTLDays > 0 {
ttlExpr = fmt.Sprintf(`TTL toDateTime(Timestamp) + toIntervalDay(%d)`, cfg.TTLDays)
}
return fmt.Sprintf(createLogsTableSQL, cfg.LogsTableName, ttlExpr)

return fmt.Sprintf(createLogsTableSQL, cfg.LogsTableName, cfg.ClusterClause(), cfg.TableEngineString(), ttlExpr)
}

func renderInsertLogsSQL(cfg *Config) string {
Expand Down
Loading