Class ClickHouseIO.Write<T>
- All Implemented Interfaces:
Serializable,HasDisplayData
- Enclosing class:
ClickHouseIO
PTransform to write to ClickHouse.- See Also:
-
Field Summary
Fields inherited from class org.apache.beam.sdk.transforms.PTransform
annotations, displayData, name, resourceHints -
Constructor Summary
Constructors -
Method Summary
Modifier and TypeMethodDescriptionexpand(PCollection<T> input) Override this method to specify how thisPTransformshould be expanded on the givenInputT.abstract Durationabstract StringjdbcUrl()abstract Durationabstract longabstract intabstract Propertiesabstract Stringtable()abstract @Nullable TableSchemawithInitialBackoff(Duration value) Set initial backoff duration.withInsertDeduplicate(Boolean value) For INSERT queries in the replicated table, specifies that deduplication of inserting blocks should be performed.If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.withInsertQuorum(@Nullable Long value) For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data.withMaxCumulativeBackoff(Duration value) Limits total time spent in backoff.withMaxInsertBlockSize(long value) The maximum block size for insertion, if we control the creation of blocks for insertion.withMaxRetries(int value) Maximum number of retries per insert.withTableSchema(@Nullable TableSchema tableSchema) Set TableSchema.Methods inherited from class org.apache.beam.sdk.transforms.PTransform
addAnnotation, compose, compose, getAdditionalInputs, getAnnotations, getDefaultOutputCoder, getDefaultOutputCoder, getDefaultOutputCoder, getKindString, getName, getResourceHints, populateDisplayData, setDisplayData, setResourceHints, toString, validate, validate
-
Constructor Details
-
Write
public Write()
-
-
Method Details
-
jdbcUrl
-
table
-
properties
-
maxInsertBlockSize
public abstract long maxInsertBlockSize() -
maxRetries
public abstract int maxRetries() -
maxCumulativeBackoff
-
initialBackoff
-
tableSchema
-
insertDistributedSync
-
insertQuorum
-
insertDeduplicate
-
expand
Description copied from class:PTransformOverride this method to specify how thisPTransformshould be expanded on the givenInputT.NOTE: This method should not be called directly. Instead apply the
PTransformshould be applied to theInputTusing theapplymethod.Composite transforms, which are defined in terms of other transforms, should return the output of one of the composed transforms. Non-composite transforms, which do not apply any transforms internally, should return a new unbound output and register evaluators (via backend-specific registration methods).
- Specified by:
expandin classPTransform<PCollection<T>,PDone>
-
withMaxInsertBlockSize
The maximum block size for insertion, if we control the creation of blocks for insertion.- Parameters:
value- number of rows- Returns:
- a
PTransformwriting data to ClickHouse - See Also:
-
withInsertDistributedSync
If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.- Parameters:
value- true to enable, null for server default- Returns:
- a
PTransformwriting data to ClickHouse
-
withInsertQuorum
For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.This setting is disabled in default server settings.
- Parameters:
value- number of replicas, 0 for disabling, null for server default- Returns:
- a
PTransformwriting data to ClickHouse - See Also:
-
withInsertDeduplicate
For INSERT queries in the replicated table, specifies that deduplication of inserting blocks should be performed.Enabled by default. Shouldn't be disabled unless your input has duplicate blocks, and you don't want to deduplicate them.
- Parameters:
value- true to enable, null for server default- Returns:
- a
PTransformwriting data to ClickHouse
-
withMaxRetries
Maximum number of retries per insert.See
FluentBackoff.withMaxRetries(int).- Parameters:
value- maximum number of retries- Returns:
- a
PTransformwriting data to ClickHouse
-
withMaxCumulativeBackoff
Limits total time spent in backoff.See
FluentBackoff.withMaxCumulativeBackoff(org.joda.time.Duration).- Parameters:
value- maximum duration- Returns:
- a
PTransformwriting data to ClickHouse
-
withInitialBackoff
Set initial backoff duration.See
FluentBackoff.withInitialBackoff(org.joda.time.Duration).- Parameters:
value- initial duration- Returns:
- a
PTransformwriting data to ClickHouse
-
withTableSchema
Set TableSchema. If not set, then TableSchema will be fetched from clickhouse server itself- Parameters:
tableSchema- schema of Table in which rows are going to be inserted- Returns:
- a
PTransformwriting data to ClickHouse
-