|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.hadoop.ozone.om.eventlistener; |
| 19 | + |
| 20 | +import java.io.IOException; |
| 21 | +import java.util.Collections; |
| 22 | +import java.util.Map; |
| 23 | +import java.util.Properties; |
| 24 | +import java.util.concurrent.TimeUnit; |
| 25 | +import org.apache.hadoop.hdds.conf.OzoneConfiguration; |
| 26 | +import org.apache.hadoop.ozone.om.helpers.OmCompletedRequestInfo; |
| 27 | +import org.apache.kafka.clients.admin.AdminClient; |
| 28 | +import org.apache.kafka.clients.admin.NewTopic; |
| 29 | +import org.apache.kafka.clients.producer.KafkaProducer; |
| 30 | +import org.apache.kafka.clients.producer.ProducerRecord; |
| 31 | +import org.slf4j.Logger; |
| 32 | +import org.slf4j.LoggerFactory; |
| 33 | + |
| 34 | +/** |
| 35 | + * This is an implementation of OMEventListener which uses the |
| 36 | + * OMEventListenerLedgerPoller as a building block to periodically poll/consume |
| 37 | + * completed operations, serialize them to a S3 schema and produce them |
| 38 | + * to a kafka topic. |
| 39 | + */ |
| 40 | +public class OMEventListenerKafkaPublisher implements OMEventListener { |
| 41 | + public static final Logger LOG = LoggerFactory.getLogger(OMEventListenerKafkaPublisher.class); |
| 42 | + |
| 43 | + private static final String KAFKA_CONFIG_PREFIX = "ozone.notify.kafka."; |
| 44 | + private static final int COMPLETED_REQUEST_CONSUMER_CORE_POOL_SIZE = 1; |
| 45 | + |
| 46 | + private OMEventListenerLedgerPoller ledgerPoller; |
| 47 | + private KafkaClientWrapper kafkaClient; |
| 48 | + private OMEventListenerLedgerPollerSeekPosition seekPosition; |
| 49 | + |
| 50 | + @Override |
| 51 | + public void initialize(OzoneConfiguration conf, OMEventListenerPluginContext pluginContext) { |
| 52 | + Map<String, String> kafkaPropsMap = conf.getPropsMatchPrefixAndTrimPrefix(KAFKA_CONFIG_PREFIX); |
| 53 | + Properties kafkaProps = new Properties(); |
| 54 | + kafkaProps.putAll(kafkaPropsMap); |
| 55 | + |
| 56 | + this.kafkaClient = new KafkaClientWrapper(kafkaProps); |
| 57 | + |
| 58 | + // TODO: these constants should be read from config |
| 59 | + long kafkaServiceInterval = 2 * 1000; |
| 60 | + long kafkaServiceTimeout = 300 * 1000; |
| 61 | + |
| 62 | + LOG.info("Creating OMEventListenerLedgerPoller with serviceInterval={}," + |
| 63 | + "serviceTimeout={}, kafkaProps={}, seekPosition={}", |
| 64 | + kafkaServiceInterval, kafkaServiceTimeout, kafkaProps, |
| 65 | + seekPosition); |
| 66 | + |
| 67 | + this.seekPosition = new OMEventListenerLedgerPollerSeekPosition(); |
| 68 | + |
| 69 | + this.ledgerPoller = new OMEventListenerLedgerPoller( |
| 70 | + kafkaServiceInterval, TimeUnit.MILLISECONDS, |
| 71 | + COMPLETED_REQUEST_CONSUMER_CORE_POOL_SIZE, |
| 72 | + kafkaServiceTimeout, pluginContext, conf, |
| 73 | + seekPosition, |
| 74 | + this::handleCompletedRequest); |
| 75 | + } |
| 76 | + |
| 77 | + @Override |
| 78 | + public void start() { |
| 79 | + ledgerPoller.start(); |
| 80 | + |
| 81 | + try { |
| 82 | + kafkaClient.initialize(); |
| 83 | + } catch (IOException ex) { |
| 84 | + LOG.error("Failure initializing kafka client", ex); |
| 85 | + } |
| 86 | + } |
| 87 | + |
| 88 | + @Override |
| 89 | + public void shutdown() { |
| 90 | + try { |
| 91 | + kafkaClient.shutdown(); |
| 92 | + } catch (IOException ex) { |
| 93 | + LOG.error("Failure shutting down kafka client", ex); |
| 94 | + } |
| 95 | + |
| 96 | + ledgerPoller.shutdown(); |
| 97 | + } |
| 98 | + |
| 99 | + // callback called by OMEventListenerLedgerPoller |
| 100 | + public void handleCompletedRequest(OmCompletedRequestInfo completedRequestInfo) { |
| 101 | + LOG.info("Processing {}", completedRequestInfo); |
| 102 | + |
| 103 | + // stub event until we implement a strategy to convert the events to |
| 104 | + // a user facing schema (e.g. S3) |
| 105 | + String event = String.format("{\"key\":\"%s/%s/%s\", \"type\":\"%s\"}", |
| 106 | + completedRequestInfo.getVolumeName(), |
| 107 | + completedRequestInfo.getBucketName(), |
| 108 | + completedRequestInfo.getKeyName(), |
| 109 | + String.valueOf(completedRequestInfo.getCmdType())); |
| 110 | + |
| 111 | + LOG.info("Sending {}", event); |
| 112 | + |
| 113 | + try { |
| 114 | + kafkaClient.send(event); |
| 115 | + } catch (IOException ex) { |
| 116 | + LOG.error("Failure to send event {}", event, ex); |
| 117 | + return; |
| 118 | + } |
| 119 | + |
| 120 | + // we can update the seek position |
| 121 | + seekPosition.set(String.valueOf(completedRequestInfo.getTrxLogIndex())); |
| 122 | + } |
| 123 | + |
| 124 | + static class KafkaClientWrapper { |
| 125 | + public static final Logger LOG = LoggerFactory.getLogger(KafkaClientWrapper.class); |
| 126 | + |
| 127 | + private final String topic; |
| 128 | + private final Properties kafkaProps; |
| 129 | + |
| 130 | + private KafkaProducer<String, String> producer; |
| 131 | + |
| 132 | + KafkaClientWrapper(Properties kafkaProps) { |
| 133 | + this.topic = (String) kafkaProps.get("topic"); |
| 134 | + this.kafkaProps = kafkaProps; |
| 135 | + } |
| 136 | + |
| 137 | + public void initialize() throws IOException { |
| 138 | + LOG.info("Initializing with properties {}", kafkaProps); |
| 139 | + this.producer = new KafkaProducer<>(kafkaProps); |
| 140 | + |
| 141 | + ensureTopicExists(); |
| 142 | + } |
| 143 | + |
| 144 | + public void shutdown() throws IOException { |
| 145 | + producer.close(); |
| 146 | + } |
| 147 | + |
| 148 | + public void send(String message) throws IOException { |
| 149 | + if (producer != null) { |
| 150 | + LOG.info("Producing event {}", message); |
| 151 | + ProducerRecord<String, String> producerRecord = |
| 152 | + new ProducerRecord<>(topic, message); |
| 153 | + producer.send(producerRecord); |
| 154 | + } else { |
| 155 | + LOG.warn("Producing event {} [KAFKA DOWN]", message); |
| 156 | + } |
| 157 | + } |
| 158 | + |
| 159 | + private void ensureTopicExists() { |
| 160 | + try (AdminClient adminClient = AdminClient.create(kafkaProps)) { |
| 161 | + LOG.info("Creating kafka topic: {}", this.topic); |
| 162 | + NewTopic newTopic = new NewTopic(this.topic, 1, (short) 1); |
| 163 | + adminClient.createTopics(Collections.singleton(newTopic)).all().get(); |
| 164 | + adminClient.close(); |
| 165 | + } catch (Exception ex) { |
| 166 | + LOG.error("Failed to create topic: {}", this.topic, ex); |
| 167 | + } |
| 168 | + } |
| 169 | + } |
| 170 | +} |
0 commit comments