/** * Called when new data arrives to the sink, and forwards it to Kafka. * * @param next The incoming data */ @Override public void invoke(IN next) throws Exception { // propagate asynchronous errors checkErroneous(); byte[] serializedKey = schema.serializeKey(next); byte[] serializedValue = schema.serializeValue(next); String targetTopic = schema.getTargetTopic(next); if (targetTopic == null) { targetTopic = defaultTopicId; } ProducerRecord<byte[], byte[]> record; if (partitioner == null) { record = new ProducerRecord<>(targetTopic, serializedKey, serializedValue); } else { record = new ProducerRecord<>( targetTopic, partitioner.partition(next, serializedKey, serializedValue, partitions.length), serializedKey, serializedValue); } if (flushOnCheckpoint) { synchronized (pendingRecordsLock) { pendingRecords++; } } producer.send(record, callback); }
/** Initializes the connection to Kafka. */ @Override public void open(Configuration configuration) { producer = getKafkaProducer(this.producerConfig); RuntimeContext ctx = getRuntimeContext(); if (partitioner != null) { partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions); } LOG.info( "Starting FlinkKafkaProducer ({}/{}) to produce into topic {}", ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), defaultTopicId); // register Kafka metrics to Flink accumulators if (!Boolean.valueOf(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { Map<MetricName, ? extends Metric> metrics = this.producer.metrics(); if (metrics == null) { // MapR's Kafka implementation returns null here. LOG.info("Producer implementation does not support metrics"); } else { for (Map.Entry<MetricName, ? extends Metric> metric : metrics.entrySet()) { String name = producerId + "-producer-" + metric.getKey().name(); DefaultKafkaMetricAccumulator kafkaAccumulator = DefaultKafkaMetricAccumulator.createFor(metric.getValue()); // best effort: we only add the accumulator if available. if (kafkaAccumulator != null) { getRuntimeContext().addAccumulator(name, kafkaAccumulator); } } } } if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { LOG.warn( "Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing."); flushOnCheckpoint = false; } if (flushOnCheckpoint) { pendingRecordsLock = new Object(); } if (logFailuresOnly) { callback = new Callback() { @Override public void onCompletion(RecordMetadata metadata, Exception e) { if (e != null) { LOG.error("Error while sending record to Kafka: " + e.getMessage(), e); } acknowledgeMessage(); } }; } else { callback = new Callback() { @Override public void onCompletion(RecordMetadata metadata, Exception exception) { if (exception != null && asyncException == null) { asyncException = exception; } acknowledgeMessage(); } }; } }