dannycranmer commented on a change in pull request #18314: URL: https://github.com/apache/flink/pull/18314#discussion_r789605310
########## File path: docs/content.zh/docs/connectors/datastream/firehose.md ########## @@ -0,0 +1,171 @@ +--- +title: Firehose +weight: 5 +type: docs +--- +<!-- +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. +--> + +# Amazon Kinesis Firehose Sink + +The Firehose sink writes to [Amazon AWS Kinesis Firehose](https://aws.amazon.com/kinesis/data-firehose/). + +Follow the instructions from the [Amazon Kinesis Firehose Developer Guide](https://docs.aws.amazon.com/firehose/latest/dev/basic-create.html) +to setup a Kinesis Data Firehose delivery stream. + +To use the connector, add the following Maven dependency to your project: + +{{< artifact flink-connector-aws-kinesis-firehose >}} + +The `KinesisFirehoseSink` uses [AWS v2 SDK for Java](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/home.html) to write data from a Flink stream into a Firehose delivery stream. + +{{< tabs "42vs28vdth5-nm76-6dz1-5m7s-5y345bu56se5u66je" >}} +{{< tab "Java" >}} +```java +KinesisFirehoseSinkElementConverter<String> elementConverter = + KinesisFirehoseSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .build(); + +Properties sinkProperties = new Properties(); +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1"); +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); + +KinesisFirehoseSink<String> kdfSink = + KinesisFirehoseSink.<String>builder() + .setKinesisClientProperties(sinkProperties) // Required + .setElementConverter(elementConverter) // Required + .setDeliveryStreamName("your-stream-name") // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(500) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(10_000) // Optional + .setMaxBatchSizeInBytes(4 * 1024 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(1000 * 1024) // Optional + .build(); + +flinkStream.sinkTo(kdfSink); +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +val elementConverter = + KinesisFirehoseSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .build() + +Properties sinkProperties = new Properties() +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1") +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id") +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key") + +val kdfSink = + KinesisFirehoseSink.<String>builder() + .setKinesisClientProperties(sinkProperties) // Required + .setElementConverter(elementConverter) // Required + .setDeliveryStreamName("your-stream-name") // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(500) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(10_000) // Optional + .setMaxBatchSizeInBytes(4 * 1024 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(1000 * 1024) // Optional + .build() + +flinkStream.sinkTo(kdfSink) +``` +{{< /tab >}} +{{< /tabs >}} + +## Configurations + +Flink's Cassandra sink is created by using the static builder `KinesisFirehoseSink.<String>builder()`. Review comment: > Cassandra Who :D (typo) ########## File path: docs/content.zh/docs/connectors/datastream/firehose.md ########## @@ -0,0 +1,171 @@ +--- +title: Firehose +weight: 5 +type: docs +--- +<!-- +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. +--> + +# Amazon Kinesis Firehose Sink + +The Firehose sink writes to [Amazon AWS Kinesis Firehose](https://aws.amazon.com/kinesis/data-firehose/). + +Follow the instructions from the [Amazon Kinesis Firehose Developer Guide](https://docs.aws.amazon.com/firehose/latest/dev/basic-create.html) +to setup a Kinesis Data Firehose delivery stream. + +To use the connector, add the following Maven dependency to your project: + +{{< artifact flink-connector-aws-kinesis-firehose >}} + +The `KinesisFirehoseSink` uses [AWS v2 SDK for Java](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/home.html) to write data from a Flink stream into a Firehose delivery stream. + +{{< tabs "42vs28vdth5-nm76-6dz1-5m7s-5y345bu56se5u66je" >}} +{{< tab "Java" >}} +```java +KinesisFirehoseSinkElementConverter<String> elementConverter = + KinesisFirehoseSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .build(); + +Properties sinkProperties = new Properties(); +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1"); +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); + +KinesisFirehoseSink<String> kdfSink = + KinesisFirehoseSink.<String>builder() + .setKinesisClientProperties(sinkProperties) // Required + .setElementConverter(elementConverter) // Required + .setDeliveryStreamName("your-stream-name") // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(500) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(10_000) // Optional + .setMaxBatchSizeInBytes(4 * 1024 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(1000 * 1024) // Optional + .build(); + +flinkStream.sinkTo(kdfSink); +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +val elementConverter = + KinesisFirehoseSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .build() + +Properties sinkProperties = new Properties() +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1") +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id") +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key") + +val kdfSink = + KinesisFirehoseSink.<String>builder() + .setKinesisClientProperties(sinkProperties) // Required + .setElementConverter(elementConverter) // Required + .setDeliveryStreamName("your-stream-name") // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(500) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(10_000) // Optional + .setMaxBatchSizeInBytes(4 * 1024 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(1000 * 1024) // Optional + .build() + +flinkStream.sinkTo(kdfSink) +``` +{{< /tab >}} +{{< /tabs >}} + +## Configurations + +Flink's Cassandra sink is created by using the static builder `KinesisFirehoseSink.<String>builder()`. + +1. __setKinesisClientProperties(Properties sinkProperties)__ + * Required. + * Supplies credentials, region and other parameter to the firehose client. +2. __setElementConverter(KinesisFirehoseSinkElementConverter elementConverter)__ + * Required. + * Supplies a serialization schema to the output. May be built using the following builder `KinesisFirehoseSinkElementConverter.<String>builder()` as per the example. +3. __setDeliveryStreamName(String deliveryStreamName)__ + * Required. + * Name of the delivery stream to sink to. +4. _setFailOnError(boolean failOnError)_ + * Optional. Default: `false`. + * Whether failed requests to write records to Firehose are treated as fatal exceptions in the sink. +5. _setMaxBatchSize(int maxBatchSize)_ + * Optional. Default: `500`. + * Maximum size of a batch to write to Firehose. +6. _setMaxInFlightRequests(int maxInFlightRequests)_ + * Optional. Default: `50`. + * The maximum number of in flight requests allowed before the sink applies backpressure. +7. _setMaxBufferedRequests(int maxBufferedRequests)_ + * Optional. Default: `10_000`. + * The maximum number of records that may be buffered in the sink before backpressure is applied. +8. _setMaxBatchSizeInBytes(int maxBatchSizeInBytes)_ + * Optional. Default: `4 * 1024 * 1024`. + * The maximum size (in bytes) a batch may become. All batches sent will be smaller than or equal to this size. +9. _setMaxTimeInBufferMS(int maxTimeInBufferMS)_ + * Optional. Default: `5000`. + * The maximum time a record may stay in the sink before being flushed. +10. _setMaxRecordSizeInBytes(int maxRecordSizeInBytes)_ + * Optional. Default: `1000 * 1024`. + * The maximum record size that the sink will accept, records larger than this will be automatically rejected. +11. _build()_ + * Constructs and returns the Firehose sink. + + +## Using Custom Kinesis Endpoints Review comment: Any reference to "Kinesis" here feels out of place. Typically when people say "Kinesis" they mean KDS ########## File path: docs/content.zh/docs/connectors/datastream/firehose.md ########## @@ -0,0 +1,171 @@ +--- +title: Firehose +weight: 5 +type: docs +--- +<!-- +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. +--> + +# Amazon Kinesis Firehose Sink + +The Firehose sink writes to [Amazon AWS Kinesis Firehose](https://aws.amazon.com/kinesis/data-firehose/). Review comment: Please update documentation reference to `Amazon Kinesis Data Firehose` ########## File path: flink-connectors/flink-connector-aws-kinesis-firehose/src/main/java/org/apache/flink/connector/firehose/sink/KinesisFirehoseException.java ########## @@ -0,0 +1,54 @@ +/* + * 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.flink.connector.firehose.sink; + +import org.apache.flink.annotation.Internal; + +/** + * A {@link RuntimeException} wrapper indicating the exception was thrown from the Kinesis Data + * Firehose Sink. + */ +@Internal Review comment: Does this bubble up to user code? If so, it is not Internal ########## File path: docs/content.zh/docs/connectors/datastream/firehose.md ########## @@ -0,0 +1,171 @@ +--- +title: Firehose +weight: 5 +type: docs +--- +<!-- +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. +--> + +# Amazon Kinesis Firehose Sink + +The Firehose sink writes to [Amazon AWS Kinesis Firehose](https://aws.amazon.com/kinesis/data-firehose/). + +Follow the instructions from the [Amazon Kinesis Firehose Developer Guide](https://docs.aws.amazon.com/firehose/latest/dev/basic-create.html) +to setup a Kinesis Data Firehose delivery stream. + +To use the connector, add the following Maven dependency to your project: + +{{< artifact flink-connector-aws-kinesis-firehose >}} + +The `KinesisFirehoseSink` uses [AWS v2 SDK for Java](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/home.html) to write data from a Flink stream into a Firehose delivery stream. + +{{< tabs "42vs28vdth5-nm76-6dz1-5m7s-5y345bu56se5u66je" >}} +{{< tab "Java" >}} +```java +KinesisFirehoseSinkElementConverter<String> elementConverter = + KinesisFirehoseSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .build(); + +Properties sinkProperties = new Properties(); +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1"); +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); + +KinesisFirehoseSink<String> kdfSink = + KinesisFirehoseSink.<String>builder() + .setKinesisClientProperties(sinkProperties) // Required + .setElementConverter(elementConverter) // Required + .setDeliveryStreamName("your-stream-name") // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(500) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(10_000) // Optional + .setMaxBatchSizeInBytes(4 * 1024 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(1000 * 1024) // Optional + .build(); + +flinkStream.sinkTo(kdfSink); +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +val elementConverter = + KinesisFirehoseSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .build() + +Properties sinkProperties = new Properties() +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1") +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id") +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key") + +val kdfSink = + KinesisFirehoseSink.<String>builder() + .setKinesisClientProperties(sinkProperties) // Required + .setElementConverter(elementConverter) // Required + .setDeliveryStreamName("your-stream-name") // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(500) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(10_000) // Optional + .setMaxBatchSizeInBytes(4 * 1024 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(1000 * 1024) // Optional + .build() + +flinkStream.sinkTo(kdfSink) +``` +{{< /tab >}} +{{< /tabs >}} + +## Configurations + +Flink's Cassandra sink is created by using the static builder `KinesisFirehoseSink.<String>builder()`. + +1. __setKinesisClientProperties(Properties sinkProperties)__ + * Required. + * Supplies credentials, region and other parameter to the firehose client. Review comment: this looks wrong, copy and paste error? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
