├── LICENSE ├── README.md ├── pom.xml └── src ├── main ├── java │ └── io │ │ └── svectors │ │ └── hbase │ │ └── cdc │ │ ├── BackpressureRetryPolicy.java │ │ ├── HbaseEndpoint.java │ │ ├── KafkaMessageProducer.java │ │ ├── KafkaProducerFactory.java │ │ ├── KafkaProps.java │ │ ├── LoggerCallback.java │ │ ├── ReplicationException.java │ │ ├── RetryPolicy.java │ │ ├── config │ │ └── KafkaConfiguration.java │ │ ├── func │ │ └── ToHRowFunction.java │ │ ├── metrics │ │ └── MetricRegistry.java │ │ ├── model │ │ └── HRow.java │ │ ├── protobuf │ │ └── generated │ │ │ ├── HColumnProtos.java │ │ │ └── HRowProtos.java │ │ ├── serde │ │ └── HRowProtobufSerde.java │ │ └── util │ │ └── TopicNameFilter.java └── resources │ └── protobuf │ ├── HColumn.proto │ └── HRow.proto └── test └── java └── io └── svectors └── hbase └── cdc ├── BaseTest.java ├── KafkaServer.java └── TestKafkaReplication.java /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "{}" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright {yyyy} {name of copyright owner} 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # hbase-connect-kafka 2 | 3 | Publish changes of HBase tables to Kafka . 4 | 5 | ## Pre-requisites 6 | 7 | * HBase 1.2.0 8 | * JDK 1.8 9 | * Kafka 0.9 10 | 11 | ## How it works 12 | 13 | [HBaseEndpoint](https://github.com/mravi/hbase-cdc-kafka/blob/master/src/main/java/io/svectors/hbase/cdc/HbaseEndpoint.java) is a custom 14 | replication end point that receives all mutations( Put / Delete). Based on the white list topics specified in the hbase-site.xml, the events are 15 | filtered and mirrored to Kafka using the Kafka producer api. 16 | 17 | ## Assumptions 18 | 19 | * Each HBase table is mapped to a Kafka topic. 20 | * HBase cluster is configured with the setting hbase.replication to true in hbase-site.xml 21 | 22 | ## Properties 23 | 24 | Have the below properties set in hbase-site.xml and add it to the HBase region server classpath. 25 | Each kafka producer property should be prefixed with `kafka`. 26 | 27 | name | data type | required | description 28 | -----|-----------|----------|------------ 29 | kafka.bootstrap.servers | string | yes | Kafka broker servers. 30 | kafka.producer.type | string | no | Can be either sync or async. Default `sync` 31 | 32 | 33 | ## Packaging 34 | 35 | * mvn clean package 36 | 37 | 38 | ## Deployment 39 | 40 | * Add hbase-cdc-kafka.jar and hbase-site.xml with the required properties to all the HBase Region servers classpath and restart them. 41 | 42 | * At HBase shell, run the following commands. 43 | 44 | ```bash 45 | hbase> create 'test', {NAME => 'd', REPLICATION_SCOPE => '1'} 46 | hbase> add_peer 'kafka-repl', ENDPOINT_CLASSNAME 'io.svectors.hbase.cdc.HbaseEndpoint' 47 | hbase> put 'test', 'r1', 'd', 'value' 48 | ``` 49 | 50 | ## TODO 51 | 52 | * Write Avro and Json Serializers for HRow. 53 | * Publish yammer metrics. 54 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 3 | 4.0.0 4 | io.svectors.hbase.cdc 5 | hbase-cdc-kafka 6 | 1.0 7 | HBase CDC to Kafka 8 | 9 | 10 | UTF-8 11 | 1.2.0 12 | 2.6.0 13 | 0.9.0.0 14 | 4.11 15 | 2.7.1 16 | 2.5.0 17 | 3.0.0 18 | 19 | 20 | 21 | 22 | junit 23 | junit 24 | ${junit.version} 25 | test 26 | 27 | 28 | org.apache.kafka 29 | kafka-clients 30 | ${kafka.version} 31 | 32 | 33 | org.apache.kafka 34 | kafka_2.10 35 | ${kafka.version} 36 | test 37 | 38 | 39 | org.apache.kafka 40 | kafka_2.10 41 | ${kafka.version} 42 | 43 | 44 | org.apache.hbase 45 | hbase-server 46 | ${hbase.version} 47 | provided 48 | 49 | 50 | com.google.protobuf 51 | protobuf-java 52 | ${protobuf.version} 53 | 54 | 55 | org.apache.hbase 56 | hbase-testing-util 57 | ${hbase.version} 58 | test 59 | 60 | 61 | 62 | 63 | 64 | compile-protobuf 65 | 66 | 67 | compile-protobuf 68 | 69 | 70 | 71 | 72 | 73 | org.apache.hadoop 74 | hadoop-maven-plugins 75 | 76 | 77 | compile-protoc 78 | generate-sources 79 | 80 | protoc 81 | 82 | 83 | ${protobuf.version} 84 | 85 | ${basedir}/src/main/resources/protobuf 86 | 87 | 88 | ${basedir}/src/main/resources/protobuf 89 | 90 | Cell.proto 91 | Row.proto 92 | 93 | 94 | ${basedir}/src/main/java/ 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | org.apache.maven.plugins 108 | maven-compiler-plugin 109 | 3.0 110 | 111 | 1.8 112 | 1.8 113 | 114 | 115 | 116 | org.apache.maven.plugins 117 | maven-shade-plugin 118 | 2.3 119 | 120 | hbase-cdc-kafka 121 | true 122 | 123 | 124 | *:* 125 | 126 | META-INF/*.SF 127 | META-INF/*.DSA 128 | META-INF/*.RSA 129 | 130 | 131 | 132 | 133 | 134 | 135 | package 136 | 137 | shade 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | 146 | 147 | 148 | 149 | 150 | 151 | 152 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/BackpressureRetryPolicy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import com.google.common.base.Throwables; 21 | import com.yammer.metrics.Metrics; 22 | import com.yammer.metrics.core.Counter; 23 | import kafka.common.QueueFullException; 24 | import org.apache.kafka.clients.producer.BufferExhaustedException; 25 | 26 | import java.time.Duration; 27 | 28 | import static io.svectors.hbase.cdc.metrics.MetricRegistry.Kafka.SEND_RETRIES; 29 | 30 | /** 31 | * @author ravi.magham 32 | */ 33 | public class BackpressureRetryPolicy implements RetryPolicy { 34 | 35 | private final Duration retryInterval = Duration.ofMillis(1000); 36 | protected final Counter retries = Metrics.newCounter(SEND_RETRIES); 37 | 38 | 39 | @Override 40 | public boolean shouldRetry(RuntimeException e) { 41 | if(e instanceof BufferExhaustedException || 42 | e instanceof QueueFullException) { 43 | // kind of applying back pressure as we make the current thread to sleep. 44 | try { 45 | Thread.sleep(retryInterval.toMillis()); 46 | retries.inc(); 47 | } catch (InterruptedException ex) { 48 | throw Throwables.propagate(ex); 49 | } 50 | return true; 51 | } else { 52 | return false; 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/HbaseEndpoint.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import static java.util.stream.Collectors.groupingBy; 21 | 22 | import java.io.IOException; 23 | import java.time.LocalDate; 24 | import java.util.List; 25 | import java.util.Map; 26 | import java.util.UUID; 27 | import io.svectors.hbase.cdc.config.KafkaConfiguration; 28 | import io.svectors.hbase.cdc.util.TopicNameFilter; 29 | import io.svectors.hbase.cdc.func.ToHRowFunction; 30 | import io.svectors.hbase.cdc.model.HRow; 31 | import org.apache.hadoop.conf.Configuration; 32 | import org.apache.hadoop.hbase.Cell; 33 | import org.apache.hadoop.hbase.CellUtil; 34 | import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; 35 | import org.apache.hadoop.hbase.wal.WAL.Entry; 36 | import org.slf4j.Logger; 37 | import org.slf4j.LoggerFactory; 38 | 39 | /** 40 | * @author ravi.magham 41 | */ 42 | public class HbaseEndpoint extends BaseReplicationEndpoint { 43 | 44 | private static final Logger LOG = LoggerFactory.getLogger(HbaseEndpoint.class); 45 | 46 | private static final ToHRowFunction TO_HROW = new ToHRowFunction(); 47 | private KafkaMessageProducer producer; 48 | private TopicNameFilter topicNameFilter; 49 | 50 | public HbaseEndpoint() { 51 | super(); 52 | } 53 | 54 | @Override 55 | public void init(Context context) throws IOException { 56 | super.init(context); 57 | LOG.info("HbaseEndpoint init: "); 58 | } 59 | 60 | @Override 61 | public UUID getPeerUUID() { 62 | return UUID.randomUUID(); 63 | } 64 | 65 | /** 66 | * 67 | * @param context 68 | * @return 69 | */ 70 | @Override 71 | public boolean replicate(ReplicateContext context) { 72 | final List entries = context.getEntries(); 73 | 74 | final Map> entriesByTable = entries.stream() 75 | .filter(entry -> topicNameFilter.test(entry.getKey().getTablename().getNameAsString())) 76 | .collect(groupingBy(entry -> entry.getKey().getTablename().getNameAsString())); 77 | 78 | // persist the data to kafka in parallel. 79 | entriesByTable.entrySet().stream().forEach(entry -> { 80 | final String tableName = entry.getKey(); 81 | final List tableEntries = entry.getValue(); 82 | 83 | tableEntries.forEach(tblEntry -> { 84 | List cells = tblEntry.getEdit().getCells(); 85 | 86 | // group the data by the rowkey. 87 | Map> columnsByRow = cells.stream() 88 | .collect(groupingBy(CellUtil::cloneRow)); 89 | 90 | // build the list of rows. 91 | columnsByRow.entrySet().stream().forEach(rowcols -> { 92 | final byte[] rowkey = rowcols.getKey(); 93 | final List columns = rowcols.getValue(); 94 | final HRow row = TO_HROW.apply(rowkey, columns); 95 | producer.send(tableName, row); 96 | }); 97 | }); 98 | }); 99 | return true; 100 | } 101 | 102 | @Override 103 | protected void doStart() { 104 | LOG.info("Hbase replication to Kafka started at " + LocalDate.now()); 105 | final Configuration hdfsConfig = ctx.getConfiguration(); 106 | final KafkaConfiguration kafkaConfig = new KafkaConfiguration(hdfsConfig); 107 | topicNameFilter = new TopicNameFilter(kafkaConfig); 108 | producer = KafkaProducerFactory.getInstance(kafkaConfig); 109 | notifyStarted(); 110 | } 111 | 112 | @Override 113 | protected void doStop() { 114 | LOG.info("Hbase replication to Kafka stopped at " + LocalDate.now()); 115 | producer.close(); 116 | notifyStopped(); 117 | } 118 | } 119 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/KafkaMessageProducer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import static io.svectors.hbase.cdc.metrics.MetricRegistry.Kafka.OVERALL_SEND_RATE; 21 | import static io.svectors.hbase.cdc.metrics.MetricRegistry.Kafka.SEND_FAILURES; 22 | 23 | import com.google.common.base.Preconditions; 24 | import com.yammer.metrics.Metrics; 25 | import com.yammer.metrics.core.Counter; 26 | import com.yammer.metrics.core.Meter; 27 | import io.svectors.hbase.cdc.config.KafkaConfiguration; 28 | import io.svectors.hbase.cdc.model.HRow; 29 | import org.apache.kafka.clients.producer.Callback; 30 | import org.apache.kafka.clients.producer.KafkaProducer; 31 | import org.apache.kafka.clients.producer.ProducerRecord; 32 | import org.apache.kafka.clients.producer.RecordMetadata; 33 | import org.slf4j.Logger; 34 | import org.slf4j.LoggerFactory; 35 | import java.io.Closeable; 36 | import java.util.concurrent.TimeUnit; 37 | 38 | 39 | /** 40 | * @author ravi.magham 41 | */ 42 | public final class KafkaMessageProducer implements Closeable { 43 | 44 | private static final Logger LOG = LoggerFactory.getLogger(KafkaMessageProducer.class); 45 | 46 | private final KafkaProducer kafkaProducer; 47 | private final RetryPolicy retryPolicy = new BackpressureRetryPolicy(); 48 | private final Meter eventsPerSec = Metrics.newMeter(OVERALL_SEND_RATE, "events", TimeUnit.SECONDS); 49 | private final Counter failures = Metrics.newCounter(SEND_FAILURES); 50 | private final boolean isAsync; 51 | private final Callback callbackHandler; 52 | 53 | 54 | /** 55 | * 56 | * @param kafkaConfiguration 57 | */ 58 | public KafkaMessageProducer(final KafkaConfiguration kafkaConfiguration) { 59 | Preconditions.checkNotNull(kafkaConfiguration); 60 | this.kafkaProducer = new KafkaProducer<>(kafkaConfiguration.getAsMap()); 61 | this.isAsync = kafkaConfiguration.isAsyncProducer(); 62 | this.callbackHandler = kafkaConfiguration.getCallbackHandler(); 63 | } 64 | 65 | /** 66 | * 67 | * @param tableName 68 | * @param row 69 | */ 70 | public void send(final String tableName, final HRow row) { 71 | final ProducerRecord record = new ProducerRecord<>(tableName, row.getRowKey(), row); 72 | if(this.isAsync) { 73 | sendAsync(record); 74 | } else { 75 | sendSync(record); 76 | } 77 | } 78 | 79 | /** 80 | * pushes the message asynchronously 81 | * @param record 82 | */ 83 | private void sendAsync(ProducerRecord record) { 84 | boolean retry = true; 85 | while(retry) { 86 | try { 87 | this.kafkaProducer.send(record, callbackHandler); 88 | eventsPerSec.mark(1); 89 | break; 90 | } catch (RuntimeException e) { 91 | retry = retryPolicy.shouldRetry(e); 92 | } catch (Exception ex) { 93 | failures.inc(); 94 | final String errorMsg = String.format("Failed to send the record to kafka topic [%s] ", record.topic()); 95 | throw new ReplicationException(errorMsg, ex); 96 | } 97 | } 98 | 99 | } 100 | 101 | /** 102 | * pushes the message synchronously 103 | * @param record 104 | */ 105 | private void sendSync(ProducerRecord record) { 106 | boolean retry = true; 107 | while(retry) { 108 | try { 109 | final RecordMetadata metadata = this.kafkaProducer.send(record).get(); 110 | eventsPerSec.mark(1); 111 | if (LOG.isDebugEnabled()) { 112 | LOG.debug(String.format(" published message to topic [%s] , partition [%s] and the next offset is [%s]", 113 | metadata.topic(), metadata.partition(), metadata.offset())); 114 | } 115 | break; 116 | } catch (RuntimeException e) { 117 | retry = retryPolicy.shouldRetry(e); 118 | } catch (Exception ex) { 119 | failures.inc(); 120 | final String errorMsg = String.format("Failed to send the record to kafka topic [%s] ", record.topic()); 121 | throw new ReplicationException(errorMsg, ex); 122 | } 123 | } 124 | } 125 | 126 | @Override 127 | public void close() { 128 | if(this.kafkaProducer != null) { 129 | this.kafkaProducer.close(); 130 | } 131 | } 132 | } 133 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/KafkaProducerFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import io.svectors.hbase.cdc.config.KafkaConfiguration; 21 | 22 | /** 23 | * @author ravi.magham 24 | */ 25 | public class KafkaProducerFactory { 26 | 27 | public static KafkaMessageProducer getInstance(final KafkaConfiguration kafkaConfiguration) { 28 | return new KafkaMessageProducer(kafkaConfiguration); 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/KafkaProps.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | 21 | /** 22 | * @author ravi.magham 23 | */ 24 | public abstract class KafkaProps { 25 | 26 | public static final String KAFKA_WHITE_LIST_TABLES_PARAM = "kafka.hbase.tables.whitelist"; 27 | public static final String KAFKA_PRODUCER_CALLBACK_HANDLER_PARAM = "kafka.callback.handler"; 28 | public static final String KAFKA_PRODUCER_TYPE_PARAM = "kafka.producer.type"; 29 | 30 | public static final String KAFKA_DEFAULT_CALLBACK_HANDLER = LoggerCallback.class.getName(); 31 | public static final String KAFKA_ASYNC_PRODUCER_TYPE = "async"; 32 | 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/LoggerCallback.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import com.yammer.metrics.Metrics; 21 | import com.yammer.metrics.core.Counter; 22 | import org.apache.kafka.clients.producer.Callback; 23 | import org.apache.kafka.clients.producer.RecordMetadata; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | import static io.svectors.hbase.cdc.metrics.MetricRegistry.Kafka.ASYNC_SEND_FAILURES; 28 | 29 | /** 30 | * @author ravi.magham 31 | */ 32 | public class LoggerCallback implements Callback { 33 | 34 | private static final Logger LOG = LoggerFactory.getLogger(LoggerCallback.class); 35 | private final Counter asyncFailures = Metrics.newCounter(ASYNC_SEND_FAILURES); 36 | 37 | @Override 38 | public void onCompletion(RecordMetadata recordMetadata, Exception e) { 39 | if (e != null) { 40 | asyncFailures.inc(); 41 | LOG.error(String.format("Exception [%s] producing to topic ", e.getMessage())); 42 | } else { 43 | LOG.info(String.format(" the topic [%s] offset is [%s] ", recordMetadata.topic(), recordMetadata.offset())); 44 | } 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/ReplicationException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | /** 21 | * @author ravi.magham 22 | */ 23 | public class ReplicationException extends RuntimeException { 24 | 25 | public ReplicationException() { 26 | super(); 27 | } 28 | 29 | public ReplicationException(String message) { 30 | super(message); 31 | } 32 | 33 | public ReplicationException(String message, Throwable cause) { 34 | super(message, cause); 35 | } 36 | 37 | public ReplicationException(Throwable cause) { 38 | super(cause); 39 | } 40 | 41 | protected ReplicationException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 42 | super(message, cause, enableSuppression, writableStackTrace); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/RetryPolicy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | /** 21 | * @author ravi.magham 22 | */ 23 | public interface RetryPolicy { 24 | 25 | boolean shouldRetry(RuntimeException e); 26 | 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/config/KafkaConfiguration.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc.config; 19 | 20 | import io.svectors.hbase.cdc.ReplicationException; 21 | import org.apache.hadoop.conf.Configuration; 22 | import org.apache.kafka.clients.producer.Callback; 23 | import java.util.Map; 24 | import java.util.Optional; 25 | 26 | import static io.svectors.hbase.cdc.KafkaProps.*; 27 | import static java.util.stream.Collectors.toMap; 28 | 29 | /** 30 | * @author ravi.magham 31 | */ 32 | public class KafkaConfiguration { 33 | 34 | public static final String KAFKA_PREFIX_KEY = "kafka."; 35 | private final Configuration configuration; 36 | private final Map kafkaProperties; 37 | 38 | /** 39 | * 40 | * @param configuration 41 | */ 42 | public KafkaConfiguration(final Configuration configuration) { 43 | this.configuration = configuration; 44 | this.kafkaProperties = filter(configuration); 45 | } 46 | 47 | /** 48 | * Filters the configuration for 49 | * @param configuration 50 | * @return 51 | */ 52 | private Map filter(final Configuration configuration) { 53 | final Map kafkaProperties = configuration.getValByRegex(KAFKA_PREFIX_KEY) 54 | .entrySet() 55 | .stream() 56 | .collect(toMap(e -> e.getKey().substring(KAFKA_PREFIX_KEY.length()), e -> e.getValue())); 57 | return kafkaProperties; 58 | } 59 | 60 | /** 61 | * Returns the properties for kafka. 62 | * @return 63 | */ 64 | public Map getAsMap() { 65 | return kafkaProperties; 66 | } 67 | 68 | /** 69 | * Returns the default callback handler when running in async mode. 70 | * @return 71 | */ 72 | public Callback getCallbackHandler() { 73 | try { 74 | final String handlerClassName = this.configuration.get(KAFKA_PRODUCER_CALLBACK_HANDLER_PARAM, 75 | KAFKA_DEFAULT_CALLBACK_HANDLER); 76 | final Class handlerClass = (Class) Class.forName(handlerClassName); 77 | return handlerClass.newInstance(); 78 | } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { 79 | throw new ReplicationException(e); 80 | } 81 | } 82 | 83 | /** 84 | * returns if we are using the sync or async producer types. 85 | * @return 86 | */ 87 | public boolean isAsyncProducer() { 88 | return KAFKA_ASYNC_PRODUCER_TYPE.equalsIgnoreCase(this.configuration.get(KAFKA_PRODUCER_TYPE_PARAM)); 89 | } 90 | 91 | /** 92 | * returns the whitelist topics if any mentioned in configuration. 93 | * @return 94 | */ 95 | public Optional getWhitelistTopics() { 96 | Object topicNames = this.configuration.get(KAFKA_WHITE_LIST_TABLES_PARAM); 97 | if(topicNames == null) { 98 | return Optional.empty(); 99 | } else { 100 | return Optional.of(topicNames.toString()); 101 | } 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/func/ToHRowFunction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc.func; 19 | 20 | import com.google.common.base.Preconditions; 21 | import io.svectors.hbase.cdc.model.HRow; 22 | import org.apache.hadoop.hbase.Cell; 23 | import org.apache.hadoop.hbase.CellUtil; 24 | import org.apache.hadoop.hbase.protobuf.generated.CellProtos; 25 | 26 | import java.util.List; 27 | import java.util.function.BiFunction; 28 | 29 | import static java.util.stream.Collectors.toList; 30 | 31 | /** 32 | * @author ravi.magham 33 | */ 34 | public class ToHRowFunction implements BiFunction, HRow> { 35 | 36 | 37 | @Override 38 | public HRow apply(byte[] rowkey, List cells) { 39 | 40 | Preconditions.checkNotNull(rowkey); 41 | Preconditions.checkNotNull(cells); 42 | final List columns = toRowColumns(cells); 43 | HRow.RowOp rowOp = null; 44 | final Cell cell = cells.get(0); 45 | final CellProtos.CellType type = CellProtos.CellType.valueOf(cell.getTypeByte()); 46 | switch (type) { 47 | case DELETE: 48 | case DELETE_COLUMN: 49 | case DELETE_FAMILY: 50 | rowOp = HRow.RowOp.DELETE; 51 | break; 52 | case PUT: 53 | rowOp = HRow.RowOp.PUT; 54 | break; 55 | } 56 | final HRow row = new HRow(rowkey, rowOp, columns); 57 | return row; 58 | } 59 | 60 | /** 61 | * maps each {@linkplain Cell} to a {@linkplain HRow.HColumn} 62 | * @param cells 63 | * @return 64 | */ 65 | private List toRowColumns(final List cells) { 66 | final List columns = cells.stream().map(cell -> { 67 | byte[] family = CellUtil.cloneFamily(cell); 68 | byte[] qualifier = CellUtil.cloneQualifier(cell); 69 | byte[] value = CellUtil.cloneValue(cell); 70 | long timestamp = cell.getTimestamp(); 71 | final HRow.HColumn column = new HRow.HColumn(family, qualifier, value, timestamp); 72 | return column; 73 | }).collect(toList()); 74 | 75 | return columns; 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/metrics/MetricRegistry.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc.metrics; 19 | 20 | import com.yammer.metrics.core.MetricName; 21 | 22 | /** 23 | * @author ravi.magham 24 | */ 25 | public abstract class MetricRegistry { 26 | 27 | public static final String GROUP = "hbase-cdc-kafka"; 28 | 29 | private MetricRegistry(){} 30 | 31 | public static abstract class Hbase { 32 | private Hbase() { 33 | } 34 | private static final String TYPE = "hbase"; 35 | 36 | public static final MetricName OVERALL_RECEIVE_RATE = 37 | new MetricName(GROUP, TYPE, "Overall receive rate"); 38 | 39 | public static final MetricName OVERALL_SEND_RATE = 40 | new MetricName(GROUP, TYPE, "Overall send rate"); 41 | 42 | public static final MetricName DROPPED_MUTATIONS = 43 | new MetricName(GROUP, TYPE, "mutations that were dropped by HBase"); 44 | 45 | } 46 | 47 | public static abstract class Kafka { 48 | private Kafka() { 49 | } 50 | private static final String TYPE = "kafka"; 51 | 52 | public static final MetricName OVERALL_SEND_RATE = 53 | new MetricName(GROUP, TYPE, "Overall Send Rate"); 54 | 55 | public static final MetricName SEND_FAILURES = 56 | new MetricName(GROUP, TYPE, "Failures to send events to kafka"); 57 | 58 | public static final MetricName SEND_RETRIES = 59 | new MetricName(GROUP, TYPE, "Retries to send events to kafka"); 60 | 61 | public static final MetricName ASYNC_SEND_FAILURES = 62 | new MetricName(GROUP, TYPE, "Failures to send events to kafka in async mode"); 63 | 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/model/HRow.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc.model; 19 | 20 | import org.apache.hadoop.hbase.HConstants; 21 | 22 | import java.util.Arrays; 23 | import java.util.List; 24 | 25 | /** 26 | * @author ravi.magham 27 | */ 28 | public class HRow { 29 | 30 | private final byte[] rowKey; 31 | 32 | private final RowOp rowOp; 33 | 34 | private final List columns; 35 | 36 | public HRow(byte[] rowKey, RowOp rowOp, HColumn... columns) { 37 | this(rowKey, rowOp, Arrays.asList(columns)); 38 | } 39 | 40 | public HRow(byte[] rowKey, RowOp rowOp, List columns) { 41 | this.rowKey = rowKey; 42 | this.rowOp = rowOp; 43 | this.columns = columns; 44 | } 45 | 46 | public byte[] getRowKey() { 47 | return rowKey; 48 | } 49 | 50 | public RowOp getRowOp() { 51 | return rowOp; 52 | } 53 | 54 | public List getColumns() { 55 | return columns; 56 | } 57 | 58 | /** 59 | * Properties for a column . 60 | */ 61 | public static class HColumn { 62 | 63 | private final byte[] family; 64 | 65 | private final long timestamp; 66 | 67 | private final byte[] qualifier; 68 | 69 | private final byte[] value; 70 | 71 | public HColumn(byte[] family, byte[] qualifier, byte[] value) { 72 | this(family,qualifier,value,HConstants.LATEST_TIMESTAMP); 73 | } 74 | 75 | public HColumn(byte[] family, byte[] qualifier, byte[] value, long timestamp) { 76 | this.family = family; 77 | this.qualifier = qualifier; 78 | this.value = value; 79 | this.timestamp = timestamp; 80 | } 81 | 82 | public byte[] getFamily() { 83 | return family; 84 | } 85 | 86 | public long getTimestamp() { 87 | return timestamp; 88 | } 89 | 90 | public byte[] getQualifier() { 91 | return qualifier; 92 | } 93 | 94 | public byte[] getValue() { 95 | return value; 96 | } 97 | } 98 | 99 | public static enum RowOp { 100 | PUT, 101 | DELETE; 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/protobuf/generated/HColumnProtos.java: -------------------------------------------------------------------------------- 1 | // Generated by the protocol buffer compiler. DO NOT EDIT! 2 | // source: HColumn.proto 3 | 4 | package io.svectors.hbase.cdc.protobuf.generated; 5 | 6 | import com.google.protobuf.AbstractMessage; 7 | 8 | public final class HColumnProtos { 9 | private HColumnProtos() {} 10 | public static void registerAllExtensions( 11 | com.google.protobuf.ExtensionRegistry registry) { 12 | } 13 | public interface HColumnOrBuilder 14 | extends com.google.protobuf.MessageOrBuilder { 15 | 16 | // optional bytes family = 1; 17 | /** 18 | * optional bytes family = 1; 19 | */ 20 | boolean hasFamily(); 21 | /** 22 | * optional bytes family = 1; 23 | */ 24 | com.google.protobuf.ByteString getFamily(); 25 | 26 | // optional bytes qualifier = 2; 27 | /** 28 | * optional bytes qualifier = 2; 29 | */ 30 | boolean hasQualifier(); 31 | /** 32 | * optional bytes qualifier = 2; 33 | */ 34 | com.google.protobuf.ByteString getQualifier(); 35 | 36 | // optional uint64 timestamp = 3; 37 | /** 38 | * optional uint64 timestamp = 3; 39 | */ 40 | boolean hasTimestamp(); 41 | /** 42 | * optional uint64 timestamp = 3; 43 | */ 44 | long getTimestamp(); 45 | 46 | // optional bytes value = 4; 47 | /** 48 | * optional bytes value = 4; 49 | */ 50 | boolean hasValue(); 51 | /** 52 | * optional bytes value = 4; 53 | */ 54 | com.google.protobuf.ByteString getValue(); 55 | } 56 | /** 57 | * Protobuf type {@code HColumn} 58 | * 59 | *
 60 |    **
 61 |    * Protocol buffer of each HColumn.
 62 |    * 
63 | */ 64 | public static final class HColumn extends 65 | com.google.protobuf.GeneratedMessage 66 | implements HColumnOrBuilder { 67 | // Use HColumn.newBuilder() to construct. 68 | private HColumn(com.google.protobuf.GeneratedMessage.Builder builder) { 69 | super(builder); 70 | this.unknownFields = builder.getUnknownFields(); 71 | } 72 | private HColumn(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } 73 | 74 | private static final HColumn defaultInstance; 75 | public static HColumn getDefaultInstance() { 76 | return defaultInstance; 77 | } 78 | 79 | public HColumn getDefaultInstanceForType() { 80 | return defaultInstance; 81 | } 82 | 83 | private final com.google.protobuf.UnknownFieldSet unknownFields; 84 | @Override 85 | public final com.google.protobuf.UnknownFieldSet 86 | getUnknownFields() { 87 | return this.unknownFields; 88 | } 89 | private HColumn( 90 | com.google.protobuf.CodedInputStream input, 91 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 92 | throws com.google.protobuf.InvalidProtocolBufferException { 93 | initFields(); 94 | int mutable_bitField0_ = 0; 95 | com.google.protobuf.UnknownFieldSet.Builder unknownFields = 96 | com.google.protobuf.UnknownFieldSet.newBuilder(); 97 | try { 98 | boolean done = false; 99 | while (!done) { 100 | int tag = input.readTag(); 101 | switch (tag) { 102 | case 0: 103 | done = true; 104 | break; 105 | default: { 106 | if (!parseUnknownField(input, unknownFields, 107 | extensionRegistry, tag)) { 108 | done = true; 109 | } 110 | break; 111 | } 112 | case 10: { 113 | bitField0_ |= 0x00000001; 114 | family_ = input.readBytes(); 115 | break; 116 | } 117 | case 18: { 118 | bitField0_ |= 0x00000002; 119 | qualifier_ = input.readBytes(); 120 | break; 121 | } 122 | case 24: { 123 | bitField0_ |= 0x00000004; 124 | timestamp_ = input.readUInt64(); 125 | break; 126 | } 127 | case 34: { 128 | bitField0_ |= 0x00000008; 129 | value_ = input.readBytes(); 130 | break; 131 | } 132 | } 133 | } 134 | } catch (com.google.protobuf.InvalidProtocolBufferException e) { 135 | throw e.setUnfinishedMessage(this); 136 | } catch (java.io.IOException e) { 137 | throw new com.google.protobuf.InvalidProtocolBufferException( 138 | e.getMessage()).setUnfinishedMessage(this); 139 | } finally { 140 | this.unknownFields = unknownFields.build(); 141 | makeExtensionsImmutable(); 142 | } 143 | } 144 | public static final com.google.protobuf.Descriptors.Descriptor 145 | getDescriptor() { 146 | return HColumnProtos.internal_static_HColumn_descriptor; 147 | } 148 | 149 | protected com.google.protobuf.GeneratedMessage.FieldAccessorTable 150 | internalGetFieldAccessorTable() { 151 | return HColumnProtos.internal_static_HColumn_fieldAccessorTable 152 | .ensureFieldAccessorsInitialized( 153 | HColumn.class, Builder.class); 154 | } 155 | 156 | public static com.google.protobuf.Parser PARSER = 157 | new com.google.protobuf.AbstractParser() { 158 | public HColumn parsePartialFrom( 159 | com.google.protobuf.CodedInputStream input, 160 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 161 | throws com.google.protobuf.InvalidProtocolBufferException { 162 | return new HColumn(input, extensionRegistry); 163 | } 164 | }; 165 | 166 | @Override 167 | public com.google.protobuf.Parser getParserForType() { 168 | return PARSER; 169 | } 170 | 171 | private int bitField0_; 172 | // optional bytes family = 1; 173 | public static final int FAMILY_FIELD_NUMBER = 1; 174 | private com.google.protobuf.ByteString family_; 175 | /** 176 | * optional bytes family = 1; 177 | */ 178 | public boolean hasFamily() { 179 | return ((bitField0_ & 0x00000001) == 0x00000001); 180 | } 181 | /** 182 | * optional bytes family = 1; 183 | */ 184 | public com.google.protobuf.ByteString getFamily() { 185 | return family_; 186 | } 187 | 188 | // optional bytes qualifier = 2; 189 | public static final int QUALIFIER_FIELD_NUMBER = 2; 190 | private com.google.protobuf.ByteString qualifier_; 191 | /** 192 | * optional bytes qualifier = 2; 193 | */ 194 | public boolean hasQualifier() { 195 | return ((bitField0_ & 0x00000002) == 0x00000002); 196 | } 197 | /** 198 | * optional bytes qualifier = 2; 199 | */ 200 | public com.google.protobuf.ByteString getQualifier() { 201 | return qualifier_; 202 | } 203 | 204 | // optional uint64 timestamp = 3; 205 | public static final int TIMESTAMP_FIELD_NUMBER = 3; 206 | private long timestamp_; 207 | /** 208 | * optional uint64 timestamp = 3; 209 | */ 210 | public boolean hasTimestamp() { 211 | return ((bitField0_ & 0x00000004) == 0x00000004); 212 | } 213 | /** 214 | * optional uint64 timestamp = 3; 215 | */ 216 | public long getTimestamp() { 217 | return timestamp_; 218 | } 219 | 220 | // optional bytes value = 4; 221 | public static final int VALUE_FIELD_NUMBER = 4; 222 | private com.google.protobuf.ByteString value_; 223 | /** 224 | * optional bytes value = 4; 225 | */ 226 | public boolean hasValue() { 227 | return ((bitField0_ & 0x00000008) == 0x00000008); 228 | } 229 | /** 230 | * optional bytes value = 4; 231 | */ 232 | public com.google.protobuf.ByteString getValue() { 233 | return value_; 234 | } 235 | 236 | private void initFields() { 237 | family_ = com.google.protobuf.ByteString.EMPTY; 238 | qualifier_ = com.google.protobuf.ByteString.EMPTY; 239 | timestamp_ = 0L; 240 | value_ = com.google.protobuf.ByteString.EMPTY; 241 | } 242 | private byte memoizedIsInitialized = -1; 243 | public final boolean isInitialized() { 244 | byte isInitialized = memoizedIsInitialized; 245 | if (isInitialized != -1) return isInitialized == 1; 246 | 247 | memoizedIsInitialized = 1; 248 | return true; 249 | } 250 | 251 | public void writeTo(com.google.protobuf.CodedOutputStream output) 252 | throws java.io.IOException { 253 | getSerializedSize(); 254 | if (((bitField0_ & 0x00000001) == 0x00000001)) { 255 | output.writeBytes(1, family_); 256 | } 257 | if (((bitField0_ & 0x00000002) == 0x00000002)) { 258 | output.writeBytes(2, qualifier_); 259 | } 260 | if (((bitField0_ & 0x00000004) == 0x00000004)) { 261 | output.writeUInt64(3, timestamp_); 262 | } 263 | if (((bitField0_ & 0x00000008) == 0x00000008)) { 264 | output.writeBytes(4, value_); 265 | } 266 | getUnknownFields().writeTo(output); 267 | } 268 | 269 | private int memoizedSerializedSize = -1; 270 | public int getSerializedSize() { 271 | int size = memoizedSerializedSize; 272 | if (size != -1) return size; 273 | 274 | size = 0; 275 | if (((bitField0_ & 0x00000001) == 0x00000001)) { 276 | size += com.google.protobuf.CodedOutputStream 277 | .computeBytesSize(1, family_); 278 | } 279 | if (((bitField0_ & 0x00000002) == 0x00000002)) { 280 | size += com.google.protobuf.CodedOutputStream 281 | .computeBytesSize(2, qualifier_); 282 | } 283 | if (((bitField0_ & 0x00000004) == 0x00000004)) { 284 | size += com.google.protobuf.CodedOutputStream 285 | .computeUInt64Size(3, timestamp_); 286 | } 287 | if (((bitField0_ & 0x00000008) == 0x00000008)) { 288 | size += com.google.protobuf.CodedOutputStream 289 | .computeBytesSize(4, value_); 290 | } 291 | size += getUnknownFields().getSerializedSize(); 292 | memoizedSerializedSize = size; 293 | return size; 294 | } 295 | 296 | private static final long serialVersionUID = 0L; 297 | @Override 298 | protected Object writeReplace() 299 | throws java.io.ObjectStreamException { 300 | return super.writeReplace(); 301 | } 302 | 303 | @Override 304 | public boolean equals(final Object obj) { 305 | if (obj == this) { 306 | return true; 307 | } 308 | if (!(obj instanceof HColumn)) { 309 | return super.equals(obj); 310 | } 311 | HColumn other = (HColumn) obj; 312 | 313 | boolean result = true; 314 | result = result && (hasFamily() == other.hasFamily()); 315 | if (hasFamily()) { 316 | result = result && getFamily() 317 | .equals(other.getFamily()); 318 | } 319 | result = result && (hasQualifier() == other.hasQualifier()); 320 | if (hasQualifier()) { 321 | result = result && getQualifier() 322 | .equals(other.getQualifier()); 323 | } 324 | result = result && (hasTimestamp() == other.hasTimestamp()); 325 | if (hasTimestamp()) { 326 | result = result && (getTimestamp() 327 | == other.getTimestamp()); 328 | } 329 | result = result && (hasValue() == other.hasValue()); 330 | if (hasValue()) { 331 | result = result && getValue() 332 | .equals(other.getValue()); 333 | } 334 | result = result && 335 | getUnknownFields().equals(other.getUnknownFields()); 336 | return result; 337 | } 338 | 339 | private int memoizedHashCode = 0; 340 | @Override 341 | public int hashCode() { 342 | if (memoizedHashCode != 0) { 343 | return memoizedHashCode; 344 | } 345 | int hash = 41; 346 | hash = (19 * hash) + getDescriptorForType().hashCode(); 347 | if (hasFamily()) { 348 | hash = (37 * hash) + FAMILY_FIELD_NUMBER; 349 | hash = (53 * hash) + getFamily().hashCode(); 350 | } 351 | if (hasQualifier()) { 352 | hash = (37 * hash) + QUALIFIER_FIELD_NUMBER; 353 | hash = (53 * hash) + getQualifier().hashCode(); 354 | } 355 | if (hasTimestamp()) { 356 | hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; 357 | hash = (53 * hash) + hashLong(getTimestamp()); 358 | } 359 | if (hasValue()) { 360 | hash = (37 * hash) + VALUE_FIELD_NUMBER; 361 | hash = (53 * hash) + getValue().hashCode(); 362 | } 363 | hash = (29 * hash) + getUnknownFields().hashCode(); 364 | memoizedHashCode = hash; 365 | return hash; 366 | } 367 | 368 | public static HColumn parseFrom( 369 | com.google.protobuf.ByteString data) 370 | throws com.google.protobuf.InvalidProtocolBufferException { 371 | return PARSER.parseFrom(data); 372 | } 373 | public static HColumn parseFrom( 374 | com.google.protobuf.ByteString data, 375 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 376 | throws com.google.protobuf.InvalidProtocolBufferException { 377 | return PARSER.parseFrom(data, extensionRegistry); 378 | } 379 | public static HColumn parseFrom(byte[] data) 380 | throws com.google.protobuf.InvalidProtocolBufferException { 381 | return PARSER.parseFrom(data); 382 | } 383 | public static HColumn parseFrom( 384 | byte[] data, 385 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 386 | throws com.google.protobuf.InvalidProtocolBufferException { 387 | return PARSER.parseFrom(data, extensionRegistry); 388 | } 389 | public static HColumn parseFrom(java.io.InputStream input) 390 | throws java.io.IOException { 391 | return PARSER.parseFrom(input); 392 | } 393 | public static HColumn parseFrom( 394 | java.io.InputStream input, 395 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 396 | throws java.io.IOException { 397 | return PARSER.parseFrom(input, extensionRegistry); 398 | } 399 | public static HColumn parseDelimitedFrom(java.io.InputStream input) 400 | throws java.io.IOException { 401 | return PARSER.parseDelimitedFrom(input); 402 | } 403 | public static HColumn parseDelimitedFrom( 404 | java.io.InputStream input, 405 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 406 | throws java.io.IOException { 407 | return PARSER.parseDelimitedFrom(input, extensionRegistry); 408 | } 409 | public static HColumn parseFrom( 410 | com.google.protobuf.CodedInputStream input) 411 | throws java.io.IOException { 412 | return PARSER.parseFrom(input); 413 | } 414 | public static HColumn parseFrom( 415 | com.google.protobuf.CodedInputStream input, 416 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 417 | throws java.io.IOException { 418 | return PARSER.parseFrom(input, extensionRegistry); 419 | } 420 | 421 | public static Builder newBuilder() { return Builder.create(); } 422 | public Builder newBuilderForType() { return newBuilder(); } 423 | public static Builder newBuilder(HColumn prototype) { 424 | return newBuilder().mergeFrom(prototype); 425 | } 426 | public Builder toBuilder() { return newBuilder(this); } 427 | 428 | @Override 429 | protected Builder newBuilderForType( 430 | com.google.protobuf.GeneratedMessage.BuilderParent parent) { 431 | Builder builder = new Builder(parent); 432 | return builder; 433 | } 434 | /** 435 | * Protobuf type {@code HColumn} 436 | * 437 | *
438 |      **
439 |      * Protocol buffer of each HColumn.
440 |      * 
441 | */ 442 | public static final class Builder extends 443 | com.google.protobuf.GeneratedMessage.Builder 444 | implements HColumnOrBuilder { 445 | public static final com.google.protobuf.Descriptors.Descriptor 446 | getDescriptor() { 447 | return HColumnProtos.internal_static_HColumn_descriptor; 448 | } 449 | 450 | protected com.google.protobuf.GeneratedMessage.FieldAccessorTable 451 | internalGetFieldAccessorTable() { 452 | return HColumnProtos.internal_static_HColumn_fieldAccessorTable 453 | .ensureFieldAccessorsInitialized( 454 | HColumn.class, Builder.class); 455 | } 456 | 457 | // Construct using com.sv.hbase.cdc.protobuf.generated.HColumnProtos.HColumn.newBuilder() 458 | private Builder() { 459 | maybeForceBuilderInitialization(); 460 | } 461 | 462 | private Builder( 463 | com.google.protobuf.GeneratedMessage.BuilderParent parent) { 464 | super(parent); 465 | maybeForceBuilderInitialization(); 466 | } 467 | private void maybeForceBuilderInitialization() { 468 | if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { 469 | } 470 | } 471 | private static Builder create() { 472 | return new Builder(); 473 | } 474 | 475 | public Builder clear() { 476 | super.clear(); 477 | family_ = com.google.protobuf.ByteString.EMPTY; 478 | bitField0_ = (bitField0_ & ~0x00000001); 479 | qualifier_ = com.google.protobuf.ByteString.EMPTY; 480 | bitField0_ = (bitField0_ & ~0x00000002); 481 | timestamp_ = 0L; 482 | bitField0_ = (bitField0_ & ~0x00000004); 483 | value_ = com.google.protobuf.ByteString.EMPTY; 484 | bitField0_ = (bitField0_ & ~0x00000008); 485 | return this; 486 | } 487 | 488 | public Builder clone() { 489 | return create().mergeFrom(buildPartial()); 490 | } 491 | 492 | public com.google.protobuf.Descriptors.Descriptor 493 | getDescriptorForType() { 494 | return HColumnProtos.internal_static_HColumn_descriptor; 495 | } 496 | 497 | public HColumn getDefaultInstanceForType() { 498 | return HColumn.getDefaultInstance(); 499 | } 500 | 501 | public HColumn build() { 502 | HColumn result = buildPartial(); 503 | if (!result.isInitialized()) { 504 | throw AbstractMessage.Builder.newUninitializedMessageException(result); 505 | } 506 | return result; 507 | } 508 | 509 | public HColumn buildPartial() { 510 | HColumn result = new HColumn(this); 511 | int from_bitField0_ = bitField0_; 512 | int to_bitField0_ = 0; 513 | if (((from_bitField0_ & 0x00000001) == 0x00000001)) { 514 | to_bitField0_ |= 0x00000001; 515 | } 516 | result.family_ = family_; 517 | if (((from_bitField0_ & 0x00000002) == 0x00000002)) { 518 | to_bitField0_ |= 0x00000002; 519 | } 520 | result.qualifier_ = qualifier_; 521 | if (((from_bitField0_ & 0x00000004) == 0x00000004)) { 522 | to_bitField0_ |= 0x00000004; 523 | } 524 | result.timestamp_ = timestamp_; 525 | if (((from_bitField0_ & 0x00000008) == 0x00000008)) { 526 | to_bitField0_ |= 0x00000008; 527 | } 528 | result.value_ = value_; 529 | result.bitField0_ = to_bitField0_; 530 | onBuilt(); 531 | return result; 532 | } 533 | 534 | public Builder mergeFrom(com.google.protobuf.Message other) { 535 | if (other instanceof HColumn) { 536 | return mergeFrom((HColumn)other); 537 | } else { 538 | super.mergeFrom(other); 539 | return this; 540 | } 541 | } 542 | 543 | public Builder mergeFrom(HColumn other) { 544 | if (other == HColumn.getDefaultInstance()) return this; 545 | if (other.hasFamily()) { 546 | setFamily(other.getFamily()); 547 | } 548 | if (other.hasQualifier()) { 549 | setQualifier(other.getQualifier()); 550 | } 551 | if (other.hasTimestamp()) { 552 | setTimestamp(other.getTimestamp()); 553 | } 554 | if (other.hasValue()) { 555 | setValue(other.getValue()); 556 | } 557 | this.mergeUnknownFields(other.getUnknownFields()); 558 | return this; 559 | } 560 | 561 | public final boolean isInitialized() { 562 | return true; 563 | } 564 | 565 | public Builder mergeFrom( 566 | com.google.protobuf.CodedInputStream input, 567 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 568 | throws java.io.IOException { 569 | HColumn parsedMessage = null; 570 | try { 571 | parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); 572 | } catch (com.google.protobuf.InvalidProtocolBufferException e) { 573 | parsedMessage = (HColumn) e.getUnfinishedMessage(); 574 | throw e; 575 | } finally { 576 | if (parsedMessage != null) { 577 | mergeFrom(parsedMessage); 578 | } 579 | } 580 | return this; 581 | } 582 | private int bitField0_; 583 | 584 | // optional bytes family = 1; 585 | private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY; 586 | /** 587 | * optional bytes family = 1; 588 | */ 589 | public boolean hasFamily() { 590 | return ((bitField0_ & 0x00000001) == 0x00000001); 591 | } 592 | /** 593 | * optional bytes family = 1; 594 | */ 595 | public com.google.protobuf.ByteString getFamily() { 596 | return family_; 597 | } 598 | /** 599 | * optional bytes family = 1; 600 | */ 601 | public Builder setFamily(com.google.protobuf.ByteString value) { 602 | if (value == null) { 603 | throw new NullPointerException(); 604 | } 605 | bitField0_ |= 0x00000001; 606 | family_ = value; 607 | onChanged(); 608 | return this; 609 | } 610 | /** 611 | * optional bytes family = 1; 612 | */ 613 | public Builder clearFamily() { 614 | bitField0_ = (bitField0_ & ~0x00000001); 615 | family_ = getDefaultInstance().getFamily(); 616 | onChanged(); 617 | return this; 618 | } 619 | 620 | // optional bytes qualifier = 2; 621 | private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY; 622 | /** 623 | * optional bytes qualifier = 2; 624 | */ 625 | public boolean hasQualifier() { 626 | return ((bitField0_ & 0x00000002) == 0x00000002); 627 | } 628 | /** 629 | * optional bytes qualifier = 2; 630 | */ 631 | public com.google.protobuf.ByteString getQualifier() { 632 | return qualifier_; 633 | } 634 | /** 635 | * optional bytes qualifier = 2; 636 | */ 637 | public Builder setQualifier(com.google.protobuf.ByteString value) { 638 | if (value == null) { 639 | throw new NullPointerException(); 640 | } 641 | bitField0_ |= 0x00000002; 642 | qualifier_ = value; 643 | onChanged(); 644 | return this; 645 | } 646 | /** 647 | * optional bytes qualifier = 2; 648 | */ 649 | public Builder clearQualifier() { 650 | bitField0_ = (bitField0_ & ~0x00000002); 651 | qualifier_ = getDefaultInstance().getQualifier(); 652 | onChanged(); 653 | return this; 654 | } 655 | 656 | // optional uint64 timestamp = 3; 657 | private long timestamp_ ; 658 | /** 659 | * optional uint64 timestamp = 3; 660 | */ 661 | public boolean hasTimestamp() { 662 | return ((bitField0_ & 0x00000004) == 0x00000004); 663 | } 664 | /** 665 | * optional uint64 timestamp = 3; 666 | */ 667 | public long getTimestamp() { 668 | return timestamp_; 669 | } 670 | /** 671 | * optional uint64 timestamp = 3; 672 | */ 673 | public Builder setTimestamp(long value) { 674 | bitField0_ |= 0x00000004; 675 | timestamp_ = value; 676 | onChanged(); 677 | return this; 678 | } 679 | /** 680 | * optional uint64 timestamp = 3; 681 | */ 682 | public Builder clearTimestamp() { 683 | bitField0_ = (bitField0_ & ~0x00000004); 684 | timestamp_ = 0L; 685 | onChanged(); 686 | return this; 687 | } 688 | 689 | // optional bytes value = 4; 690 | private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY; 691 | /** 692 | * optional bytes value = 4; 693 | */ 694 | public boolean hasValue() { 695 | return ((bitField0_ & 0x00000008) == 0x00000008); 696 | } 697 | /** 698 | * optional bytes value = 4; 699 | */ 700 | public com.google.protobuf.ByteString getValue() { 701 | return value_; 702 | } 703 | /** 704 | * optional bytes value = 4; 705 | */ 706 | public Builder setValue(com.google.protobuf.ByteString value) { 707 | if (value == null) { 708 | throw new NullPointerException(); 709 | } 710 | bitField0_ |= 0x00000008; 711 | value_ = value; 712 | onChanged(); 713 | return this; 714 | } 715 | /** 716 | * optional bytes value = 4; 717 | */ 718 | public Builder clearValue() { 719 | bitField0_ = (bitField0_ & ~0x00000008); 720 | value_ = getDefaultInstance().getValue(); 721 | onChanged(); 722 | return this; 723 | } 724 | 725 | // @@protoc_insertion_point(builder_scope:HColumn) 726 | } 727 | 728 | static { 729 | defaultInstance = new HColumn(true); 730 | defaultInstance.initFields(); 731 | } 732 | 733 | // @@protoc_insertion_point(class_scope:HColumn) 734 | } 735 | 736 | private static com.google.protobuf.Descriptors.Descriptor 737 | internal_static_HColumn_descriptor; 738 | private static 739 | com.google.protobuf.GeneratedMessage.FieldAccessorTable 740 | internal_static_HColumn_fieldAccessorTable; 741 | 742 | public static com.google.protobuf.Descriptors.FileDescriptor 743 | getDescriptor() { 744 | return descriptor; 745 | } 746 | private static com.google.protobuf.Descriptors.FileDescriptor 747 | descriptor; 748 | static { 749 | String[] descriptorData = { 750 | "\n\rHColumn.proto\"N\n\007HColumn\022\016\n\006family\030\001 \001" + 751 | "(\014\022\021\n\tqualifier\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004" + 752 | "\022\r\n\005value\030\004 \001(\014B9\n#com.sv.hbase.cdc.prot" + 753 | "obuf.generatedB\rHColumnProtosH\001\240\001\001" 754 | }; 755 | com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = 756 | new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { 757 | public com.google.protobuf.ExtensionRegistry assignDescriptors( 758 | com.google.protobuf.Descriptors.FileDescriptor root) { 759 | descriptor = root; 760 | internal_static_HColumn_descriptor = 761 | getDescriptor().getMessageTypes().get(0); 762 | internal_static_HColumn_fieldAccessorTable = new 763 | com.google.protobuf.GeneratedMessage.FieldAccessorTable( 764 | internal_static_HColumn_descriptor, 765 | new String[] { "Family", "Qualifier", "Timestamp", "Value", }); 766 | return null; 767 | } 768 | }; 769 | com.google.protobuf.Descriptors.FileDescriptor 770 | .internalBuildGeneratedFileFrom(descriptorData, 771 | new com.google.protobuf.Descriptors.FileDescriptor[] { 772 | }, assigner); 773 | } 774 | 775 | // @@protoc_insertion_point(outer_class_scope) 776 | } 777 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/protobuf/generated/HRowProtos.java: -------------------------------------------------------------------------------- 1 | // Generated by the protocol buffer compiler. DO NOT EDIT! 2 | // source: HRow.proto 3 | 4 | package io.svectors.hbase.cdc.protobuf.generated; 5 | 6 | import com.google.protobuf.AbstractMessage; 7 | 8 | public final class HRowProtos { 9 | private HRowProtos() {} 10 | public static void registerAllExtensions( 11 | com.google.protobuf.ExtensionRegistry registry) { 12 | } 13 | /** 14 | * Protobuf enum {@code RowOp} 15 | * 16 | *
  17 |    **
  18 |    * The row operation . for now , it is either Put or delete.
  19 |    * 
20 | */ 21 | public enum RowOp 22 | implements com.google.protobuf.ProtocolMessageEnum { 23 | /** 24 | * PUT = 1; 25 | */ 26 | PUT(0, 1), 27 | /** 28 | * DELETE = 2; 29 | */ 30 | DELETE(1, 2), 31 | ; 32 | 33 | /** 34 | * PUT = 1; 35 | */ 36 | public static final int PUT_VALUE = 1; 37 | /** 38 | * DELETE = 2; 39 | */ 40 | public static final int DELETE_VALUE = 2; 41 | 42 | 43 | public final int getNumber() { return value; } 44 | 45 | public static RowOp valueOf(int value) { 46 | switch (value) { 47 | case 1: return PUT; 48 | case 2: return DELETE; 49 | default: return null; 50 | } 51 | } 52 | 53 | public static com.google.protobuf.Internal.EnumLiteMap 54 | internalGetValueMap() { 55 | return internalValueMap; 56 | } 57 | private static com.google.protobuf.Internal.EnumLiteMap 58 | internalValueMap = 59 | new com.google.protobuf.Internal.EnumLiteMap() { 60 | public RowOp findValueByNumber(int number) { 61 | return RowOp.valueOf(number); 62 | } 63 | }; 64 | 65 | public final com.google.protobuf.Descriptors.EnumValueDescriptor 66 | getValueDescriptor() { 67 | return getDescriptor().getValues().get(index); 68 | } 69 | public final com.google.protobuf.Descriptors.EnumDescriptor 70 | getDescriptorForType() { 71 | return getDescriptor(); 72 | } 73 | public static final com.google.protobuf.Descriptors.EnumDescriptor 74 | getDescriptor() { 75 | return HRowProtos.getDescriptor().getEnumTypes().get(0); 76 | } 77 | 78 | private static final RowOp[] VALUES = values(); 79 | 80 | public static RowOp valueOf( 81 | com.google.protobuf.Descriptors.EnumValueDescriptor desc) { 82 | if (desc.getType() != getDescriptor()) { 83 | throw new IllegalArgumentException( 84 | "EnumValueDescriptor is not for this type."); 85 | } 86 | return VALUES[desc.getIndex()]; 87 | } 88 | 89 | private final int index; 90 | private final int value; 91 | 92 | private RowOp(int index, int value) { 93 | this.index = index; 94 | this.value = value; 95 | } 96 | 97 | // @@protoc_insertion_point(enum_scope:RowOp) 98 | } 99 | 100 | public interface RowOrBuilder 101 | extends com.google.protobuf.MessageOrBuilder { 102 | 103 | // optional bytes row = 1; 104 | /** 105 | * optional bytes row = 1; 106 | */ 107 | boolean hasRow(); 108 | /** 109 | * optional bytes row = 1; 110 | */ 111 | com.google.protobuf.ByteString getRow(); 112 | 113 | // optional .RowOp op = 2; 114 | /** 115 | * optional .RowOp op = 2; 116 | */ 117 | boolean hasOp(); 118 | /** 119 | * optional .RowOp op = 2; 120 | */ 121 | RowOp getOp(); 122 | 123 | // repeated .HColumn column = 3; 124 | /** 125 | * repeated .HColumn column = 3; 126 | */ 127 | java.util.List 128 | getColumnList(); 129 | /** 130 | * repeated .HColumn column = 3; 131 | */ 132 | HColumnProtos.HColumn getColumn(int index); 133 | /** 134 | * repeated .HColumn column = 3; 135 | */ 136 | int getColumnCount(); 137 | /** 138 | * repeated .HColumn column = 3; 139 | */ 140 | java.util.List 141 | getColumnOrBuilderList(); 142 | /** 143 | * repeated .HColumn column = 3; 144 | */ 145 | HColumnProtos.HColumnOrBuilder getColumnOrBuilder( 146 | int index); 147 | } 148 | /** 149 | * Protobuf type {@code HRow} 150 | * 151 | *
 152 |    **
 153 |    * Protocol buffer of HRow.
 154 |    * 
155 | */ 156 | public static final class Row extends 157 | com.google.protobuf.GeneratedMessage 158 | implements RowOrBuilder { 159 | // Use HRow.newBuilder() to construct. 160 | private Row(com.google.protobuf.GeneratedMessage.Builder builder) { 161 | super(builder); 162 | this.unknownFields = builder.getUnknownFields(); 163 | } 164 | private Row(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } 165 | 166 | private static final Row defaultInstance; 167 | public static Row getDefaultInstance() { 168 | return defaultInstance; 169 | } 170 | 171 | public Row getDefaultInstanceForType() { 172 | return defaultInstance; 173 | } 174 | 175 | private final com.google.protobuf.UnknownFieldSet unknownFields; 176 | @Override 177 | public final com.google.protobuf.UnknownFieldSet 178 | getUnknownFields() { 179 | return this.unknownFields; 180 | } 181 | private Row( 182 | com.google.protobuf.CodedInputStream input, 183 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 184 | throws com.google.protobuf.InvalidProtocolBufferException { 185 | initFields(); 186 | int mutable_bitField0_ = 0; 187 | com.google.protobuf.UnknownFieldSet.Builder unknownFields = 188 | com.google.protobuf.UnknownFieldSet.newBuilder(); 189 | try { 190 | boolean done = false; 191 | while (!done) { 192 | int tag = input.readTag(); 193 | switch (tag) { 194 | case 0: 195 | done = true; 196 | break; 197 | default: { 198 | if (!parseUnknownField(input, unknownFields, 199 | extensionRegistry, tag)) { 200 | done = true; 201 | } 202 | break; 203 | } 204 | case 10: { 205 | bitField0_ |= 0x00000001; 206 | row_ = input.readBytes(); 207 | break; 208 | } 209 | case 16: { 210 | int rawValue = input.readEnum(); 211 | RowOp value = RowOp.valueOf(rawValue); 212 | if (value == null) { 213 | unknownFields.mergeVarintField(2, rawValue); 214 | } else { 215 | bitField0_ |= 0x00000002; 216 | op_ = value; 217 | } 218 | break; 219 | } 220 | case 26: { 221 | if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) { 222 | column_ = new java.util.ArrayList(); 223 | mutable_bitField0_ |= 0x00000004; 224 | } 225 | column_.add(input.readMessage(HColumnProtos.HColumn.PARSER, extensionRegistry)); 226 | break; 227 | } 228 | } 229 | } 230 | } catch (com.google.protobuf.InvalidProtocolBufferException e) { 231 | throw e.setUnfinishedMessage(this); 232 | } catch (java.io.IOException e) { 233 | throw new com.google.protobuf.InvalidProtocolBufferException( 234 | e.getMessage()).setUnfinishedMessage(this); 235 | } finally { 236 | if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) { 237 | column_ = java.util.Collections.unmodifiableList(column_); 238 | } 239 | this.unknownFields = unknownFields.build(); 240 | makeExtensionsImmutable(); 241 | } 242 | } 243 | public static final com.google.protobuf.Descriptors.Descriptor 244 | getDescriptor() { 245 | return HRowProtos.internal_static_Row_descriptor; 246 | } 247 | 248 | protected com.google.protobuf.GeneratedMessage.FieldAccessorTable 249 | internalGetFieldAccessorTable() { 250 | return HRowProtos.internal_static_Row_fieldAccessorTable 251 | .ensureFieldAccessorsInitialized( 252 | Row.class, Builder.class); 253 | } 254 | 255 | public static com.google.protobuf.Parser PARSER = 256 | new com.google.protobuf.AbstractParser() { 257 | public Row parsePartialFrom( 258 | com.google.protobuf.CodedInputStream input, 259 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 260 | throws com.google.protobuf.InvalidProtocolBufferException { 261 | return new Row(input, extensionRegistry); 262 | } 263 | }; 264 | 265 | @Override 266 | public com.google.protobuf.Parser getParserForType() { 267 | return PARSER; 268 | } 269 | 270 | private int bitField0_; 271 | // optional bytes row = 1; 272 | public static final int ROW_FIELD_NUMBER = 1; 273 | private com.google.protobuf.ByteString row_; 274 | /** 275 | * optional bytes row = 1; 276 | */ 277 | public boolean hasRow() { 278 | return ((bitField0_ & 0x00000001) == 0x00000001); 279 | } 280 | /** 281 | * optional bytes row = 1; 282 | */ 283 | public com.google.protobuf.ByteString getRow() { 284 | return row_; 285 | } 286 | 287 | // optional .RowOp op = 2; 288 | public static final int OP_FIELD_NUMBER = 2; 289 | private RowOp op_; 290 | /** 291 | * optional .RowOp op = 2; 292 | */ 293 | public boolean hasOp() { 294 | return ((bitField0_ & 0x00000002) == 0x00000002); 295 | } 296 | /** 297 | * optional .RowOp op = 2; 298 | */ 299 | public RowOp getOp() { 300 | return op_; 301 | } 302 | 303 | // repeated .HColumn column = 3; 304 | public static final int COLUMN_FIELD_NUMBER = 3; 305 | private java.util.List column_; 306 | /** 307 | * repeated .HColumn column = 3; 308 | */ 309 | public java.util.List getColumnList() { 310 | return column_; 311 | } 312 | /** 313 | * repeated .HColumn column = 3; 314 | */ 315 | public java.util.List 316 | getColumnOrBuilderList() { 317 | return column_; 318 | } 319 | /** 320 | * repeated .HColumn column = 3; 321 | */ 322 | public int getColumnCount() { 323 | return column_.size(); 324 | } 325 | /** 326 | * repeated .HColumn column = 3; 327 | */ 328 | public HColumnProtos.HColumn getColumn(int index) { 329 | return column_.get(index); 330 | } 331 | /** 332 | * repeated .HColumn column = 3; 333 | */ 334 | public HColumnProtos.HColumnOrBuilder getColumnOrBuilder( 335 | int index) { 336 | return column_.get(index); 337 | } 338 | 339 | private void initFields() { 340 | row_ = com.google.protobuf.ByteString.EMPTY; 341 | op_ = RowOp.PUT; 342 | column_ = java.util.Collections.emptyList(); 343 | } 344 | private byte memoizedIsInitialized = -1; 345 | public final boolean isInitialized() { 346 | byte isInitialized = memoizedIsInitialized; 347 | if (isInitialized != -1) return isInitialized == 1; 348 | 349 | memoizedIsInitialized = 1; 350 | return true; 351 | } 352 | 353 | public void writeTo(com.google.protobuf.CodedOutputStream output) 354 | throws java.io.IOException { 355 | getSerializedSize(); 356 | if (((bitField0_ & 0x00000001) == 0x00000001)) { 357 | output.writeBytes(1, row_); 358 | } 359 | if (((bitField0_ & 0x00000002) == 0x00000002)) { 360 | output.writeEnum(2, op_.getNumber()); 361 | } 362 | for (int i = 0; i < column_.size(); i++) { 363 | output.writeMessage(3, column_.get(i)); 364 | } 365 | getUnknownFields().writeTo(output); 366 | } 367 | 368 | private int memoizedSerializedSize = -1; 369 | public int getSerializedSize() { 370 | int size = memoizedSerializedSize; 371 | if (size != -1) return size; 372 | 373 | size = 0; 374 | if (((bitField0_ & 0x00000001) == 0x00000001)) { 375 | size += com.google.protobuf.CodedOutputStream 376 | .computeBytesSize(1, row_); 377 | } 378 | if (((bitField0_ & 0x00000002) == 0x00000002)) { 379 | size += com.google.protobuf.CodedOutputStream 380 | .computeEnumSize(2, op_.getNumber()); 381 | } 382 | for (int i = 0; i < column_.size(); i++) { 383 | size += com.google.protobuf.CodedOutputStream 384 | .computeMessageSize(3, column_.get(i)); 385 | } 386 | size += getUnknownFields().getSerializedSize(); 387 | memoizedSerializedSize = size; 388 | return size; 389 | } 390 | 391 | private static final long serialVersionUID = 0L; 392 | @Override 393 | protected Object writeReplace() 394 | throws java.io.ObjectStreamException { 395 | return super.writeReplace(); 396 | } 397 | 398 | @Override 399 | public boolean equals(final Object obj) { 400 | if (obj == this) { 401 | return true; 402 | } 403 | if (!(obj instanceof Row)) { 404 | return super.equals(obj); 405 | } 406 | Row other = (Row) obj; 407 | 408 | boolean result = true; 409 | result = result && (hasRow() == other.hasRow()); 410 | if (hasRow()) { 411 | result = result && getRow() 412 | .equals(other.getRow()); 413 | } 414 | result = result && (hasOp() == other.hasOp()); 415 | if (hasOp()) { 416 | result = result && 417 | (getOp() == other.getOp()); 418 | } 419 | result = result && getColumnList() 420 | .equals(other.getColumnList()); 421 | result = result && 422 | getUnknownFields().equals(other.getUnknownFields()); 423 | return result; 424 | } 425 | 426 | private int memoizedHashCode = 0; 427 | @Override 428 | public int hashCode() { 429 | if (memoizedHashCode != 0) { 430 | return memoizedHashCode; 431 | } 432 | int hash = 41; 433 | hash = (19 * hash) + getDescriptorForType().hashCode(); 434 | if (hasRow()) { 435 | hash = (37 * hash) + ROW_FIELD_NUMBER; 436 | hash = (53 * hash) + getRow().hashCode(); 437 | } 438 | if (hasOp()) { 439 | hash = (37 * hash) + OP_FIELD_NUMBER; 440 | hash = (53 * hash) + hashEnum(getOp()); 441 | } 442 | if (getColumnCount() > 0) { 443 | hash = (37 * hash) + COLUMN_FIELD_NUMBER; 444 | hash = (53 * hash) + getColumnList().hashCode(); 445 | } 446 | hash = (29 * hash) + getUnknownFields().hashCode(); 447 | memoizedHashCode = hash; 448 | return hash; 449 | } 450 | 451 | public static Row parseFrom( 452 | com.google.protobuf.ByteString data) 453 | throws com.google.protobuf.InvalidProtocolBufferException { 454 | return PARSER.parseFrom(data); 455 | } 456 | public static Row parseFrom( 457 | com.google.protobuf.ByteString data, 458 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 459 | throws com.google.protobuf.InvalidProtocolBufferException { 460 | return PARSER.parseFrom(data, extensionRegistry); 461 | } 462 | public static Row parseFrom(byte[] data) 463 | throws com.google.protobuf.InvalidProtocolBufferException { 464 | return PARSER.parseFrom(data); 465 | } 466 | public static Row parseFrom( 467 | byte[] data, 468 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 469 | throws com.google.protobuf.InvalidProtocolBufferException { 470 | return PARSER.parseFrom(data, extensionRegistry); 471 | } 472 | public static Row parseFrom(java.io.InputStream input) 473 | throws java.io.IOException { 474 | return PARSER.parseFrom(input); 475 | } 476 | public static Row parseFrom( 477 | java.io.InputStream input, 478 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 479 | throws java.io.IOException { 480 | return PARSER.parseFrom(input, extensionRegistry); 481 | } 482 | public static Row parseDelimitedFrom(java.io.InputStream input) 483 | throws java.io.IOException { 484 | return PARSER.parseDelimitedFrom(input); 485 | } 486 | public static Row parseDelimitedFrom( 487 | java.io.InputStream input, 488 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 489 | throws java.io.IOException { 490 | return PARSER.parseDelimitedFrom(input, extensionRegistry); 491 | } 492 | public static Row parseFrom( 493 | com.google.protobuf.CodedInputStream input) 494 | throws java.io.IOException { 495 | return PARSER.parseFrom(input); 496 | } 497 | public static Row parseFrom( 498 | com.google.protobuf.CodedInputStream input, 499 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 500 | throws java.io.IOException { 501 | return PARSER.parseFrom(input, extensionRegistry); 502 | } 503 | 504 | public static Builder newBuilder() { return Builder.create(); } 505 | public Builder newBuilderForType() { return newBuilder(); } 506 | public static Builder newBuilder(Row prototype) { 507 | return newBuilder().mergeFrom(prototype); 508 | } 509 | public Builder toBuilder() { return newBuilder(this); } 510 | 511 | @Override 512 | protected Builder newBuilderForType( 513 | com.google.protobuf.GeneratedMessage.BuilderParent parent) { 514 | Builder builder = new Builder(parent); 515 | return builder; 516 | } 517 | /** 518 | * Protobuf type {@code HRow} 519 | * 520 | *
 521 |      **
 522 |      * Protocol buffer of HRow.
 523 |      * 
524 | */ 525 | public static final class Builder extends 526 | com.google.protobuf.GeneratedMessage.Builder 527 | implements RowOrBuilder { 528 | public static final com.google.protobuf.Descriptors.Descriptor 529 | getDescriptor() { 530 | return HRowProtos.internal_static_Row_descriptor; 531 | } 532 | 533 | protected com.google.protobuf.GeneratedMessage.FieldAccessorTable 534 | internalGetFieldAccessorTable() { 535 | return HRowProtos.internal_static_Row_fieldAccessorTable 536 | .ensureFieldAccessorsInitialized( 537 | Row.class, Builder.class); 538 | } 539 | 540 | // Construct using com.sv.hbase.cdc.protobuf.generated.HRowProtos.HRow.newBuilder() 541 | private Builder() { 542 | maybeForceBuilderInitialization(); 543 | } 544 | 545 | private Builder( 546 | com.google.protobuf.GeneratedMessage.BuilderParent parent) { 547 | super(parent); 548 | maybeForceBuilderInitialization(); 549 | } 550 | private void maybeForceBuilderInitialization() { 551 | if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { 552 | getColumnFieldBuilder(); 553 | } 554 | } 555 | private static Builder create() { 556 | return new Builder(); 557 | } 558 | 559 | public Builder clear() { 560 | super.clear(); 561 | row_ = com.google.protobuf.ByteString.EMPTY; 562 | bitField0_ = (bitField0_ & ~0x00000001); 563 | op_ = RowOp.PUT; 564 | bitField0_ = (bitField0_ & ~0x00000002); 565 | if (columnBuilder_ == null) { 566 | column_ = java.util.Collections.emptyList(); 567 | bitField0_ = (bitField0_ & ~0x00000004); 568 | } else { 569 | columnBuilder_.clear(); 570 | } 571 | return this; 572 | } 573 | 574 | public Builder clone() { 575 | return create().mergeFrom(buildPartial()); 576 | } 577 | 578 | public com.google.protobuf.Descriptors.Descriptor 579 | getDescriptorForType() { 580 | return HRowProtos.internal_static_Row_descriptor; 581 | } 582 | 583 | public Row getDefaultInstanceForType() { 584 | return Row.getDefaultInstance(); 585 | } 586 | 587 | public Row build() { 588 | Row result = buildPartial(); 589 | if (!result.isInitialized()) { 590 | throw AbstractMessage.Builder.newUninitializedMessageException(result); 591 | } 592 | return result; 593 | } 594 | 595 | public Row buildPartial() { 596 | Row result = new Row(this); 597 | int from_bitField0_ = bitField0_; 598 | int to_bitField0_ = 0; 599 | if (((from_bitField0_ & 0x00000001) == 0x00000001)) { 600 | to_bitField0_ |= 0x00000001; 601 | } 602 | result.row_ = row_; 603 | if (((from_bitField0_ & 0x00000002) == 0x00000002)) { 604 | to_bitField0_ |= 0x00000002; 605 | } 606 | result.op_ = op_; 607 | if (columnBuilder_ == null) { 608 | if (((bitField0_ & 0x00000004) == 0x00000004)) { 609 | column_ = java.util.Collections.unmodifiableList(column_); 610 | bitField0_ = (bitField0_ & ~0x00000004); 611 | } 612 | result.column_ = column_; 613 | } else { 614 | result.column_ = columnBuilder_.build(); 615 | } 616 | result.bitField0_ = to_bitField0_; 617 | onBuilt(); 618 | return result; 619 | } 620 | 621 | public Builder mergeFrom(com.google.protobuf.Message other) { 622 | if (other instanceof Row) { 623 | return mergeFrom((Row)other); 624 | } else { 625 | super.mergeFrom(other); 626 | return this; 627 | } 628 | } 629 | 630 | public Builder mergeFrom(Row other) { 631 | if (other == Row.getDefaultInstance()) return this; 632 | if (other.hasRow()) { 633 | setRow(other.getRow()); 634 | } 635 | if (other.hasOp()) { 636 | setOp(other.getOp()); 637 | } 638 | if (columnBuilder_ == null) { 639 | if (!other.column_.isEmpty()) { 640 | if (column_.isEmpty()) { 641 | column_ = other.column_; 642 | bitField0_ = (bitField0_ & ~0x00000004); 643 | } else { 644 | ensureColumnIsMutable(); 645 | column_.addAll(other.column_); 646 | } 647 | onChanged(); 648 | } 649 | } else { 650 | if (!other.column_.isEmpty()) { 651 | if (columnBuilder_.isEmpty()) { 652 | columnBuilder_.dispose(); 653 | columnBuilder_ = null; 654 | column_ = other.column_; 655 | bitField0_ = (bitField0_ & ~0x00000004); 656 | columnBuilder_ = 657 | com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? 658 | getColumnFieldBuilder() : null; 659 | } else { 660 | columnBuilder_.addAllMessages(other.column_); 661 | } 662 | } 663 | } 664 | this.mergeUnknownFields(other.getUnknownFields()); 665 | return this; 666 | } 667 | 668 | public final boolean isInitialized() { 669 | return true; 670 | } 671 | 672 | public Builder mergeFrom( 673 | com.google.protobuf.CodedInputStream input, 674 | com.google.protobuf.ExtensionRegistryLite extensionRegistry) 675 | throws java.io.IOException { 676 | Row parsedMessage = null; 677 | try { 678 | parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); 679 | } catch (com.google.protobuf.InvalidProtocolBufferException e) { 680 | parsedMessage = (Row) e.getUnfinishedMessage(); 681 | throw e; 682 | } finally { 683 | if (parsedMessage != null) { 684 | mergeFrom(parsedMessage); 685 | } 686 | } 687 | return this; 688 | } 689 | private int bitField0_; 690 | 691 | // optional bytes row = 1; 692 | private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY; 693 | /** 694 | * optional bytes row = 1; 695 | */ 696 | public boolean hasRow() { 697 | return ((bitField0_ & 0x00000001) == 0x00000001); 698 | } 699 | /** 700 | * optional bytes row = 1; 701 | */ 702 | public com.google.protobuf.ByteString getRow() { 703 | return row_; 704 | } 705 | /** 706 | * optional bytes row = 1; 707 | */ 708 | public Builder setRow(com.google.protobuf.ByteString value) { 709 | if (value == null) { 710 | throw new NullPointerException(); 711 | } 712 | bitField0_ |= 0x00000001; 713 | row_ = value; 714 | onChanged(); 715 | return this; 716 | } 717 | /** 718 | * optional bytes row = 1; 719 | */ 720 | public Builder clearRow() { 721 | bitField0_ = (bitField0_ & ~0x00000001); 722 | row_ = getDefaultInstance().getRow(); 723 | onChanged(); 724 | return this; 725 | } 726 | 727 | // optional .RowOp op = 2; 728 | private RowOp op_ = RowOp.PUT; 729 | /** 730 | * optional .RowOp op = 2; 731 | */ 732 | public boolean hasOp() { 733 | return ((bitField0_ & 0x00000002) == 0x00000002); 734 | } 735 | /** 736 | * optional .RowOp op = 2; 737 | */ 738 | public RowOp getOp() { 739 | return op_; 740 | } 741 | /** 742 | * optional .RowOp op = 2; 743 | */ 744 | public Builder setOp(RowOp value) { 745 | if (value == null) { 746 | throw new NullPointerException(); 747 | } 748 | bitField0_ |= 0x00000002; 749 | op_ = value; 750 | onChanged(); 751 | return this; 752 | } 753 | /** 754 | * optional .RowOp op = 2; 755 | */ 756 | public Builder clearOp() { 757 | bitField0_ = (bitField0_ & ~0x00000002); 758 | op_ = RowOp.PUT; 759 | onChanged(); 760 | return this; 761 | } 762 | 763 | // repeated .HColumn column = 3; 764 | private java.util.List column_ = 765 | java.util.Collections.emptyList(); 766 | private void ensureColumnIsMutable() { 767 | if (!((bitField0_ & 0x00000004) == 0x00000004)) { 768 | column_ = new java.util.ArrayList(column_); 769 | bitField0_ |= 0x00000004; 770 | } 771 | } 772 | 773 | private com.google.protobuf.RepeatedFieldBuilder< 774 | HColumnProtos.HColumn, HColumnProtos.HColumn.Builder, HColumnProtos.HColumnOrBuilder> columnBuilder_; 775 | 776 | /** 777 | * repeated .HColumn column = 3; 778 | */ 779 | public java.util.List getColumnList() { 780 | if (columnBuilder_ == null) { 781 | return java.util.Collections.unmodifiableList(column_); 782 | } else { 783 | return columnBuilder_.getMessageList(); 784 | } 785 | } 786 | /** 787 | * repeated .HColumn column = 3; 788 | */ 789 | public int getColumnCount() { 790 | if (columnBuilder_ == null) { 791 | return column_.size(); 792 | } else { 793 | return columnBuilder_.getCount(); 794 | } 795 | } 796 | /** 797 | * repeated .HColumn column = 3; 798 | */ 799 | public HColumnProtos.HColumn getColumn(int index) { 800 | if (columnBuilder_ == null) { 801 | return column_.get(index); 802 | } else { 803 | return columnBuilder_.getMessage(index); 804 | } 805 | } 806 | /** 807 | * repeated .HColumn column = 3; 808 | */ 809 | public Builder setColumn( 810 | int index, HColumnProtos.HColumn value) { 811 | if (columnBuilder_ == null) { 812 | if (value == null) { 813 | throw new NullPointerException(); 814 | } 815 | ensureColumnIsMutable(); 816 | column_.set(index, value); 817 | onChanged(); 818 | } else { 819 | columnBuilder_.setMessage(index, value); 820 | } 821 | return this; 822 | } 823 | /** 824 | * repeated .HColumn column = 3; 825 | */ 826 | public Builder setColumn( 827 | int index, HColumnProtos.HColumn.Builder builderForValue) { 828 | if (columnBuilder_ == null) { 829 | ensureColumnIsMutable(); 830 | column_.set(index, builderForValue.build()); 831 | onChanged(); 832 | } else { 833 | columnBuilder_.setMessage(index, builderForValue.build()); 834 | } 835 | return this; 836 | } 837 | /** 838 | * repeated .HColumn column = 3; 839 | */ 840 | public Builder addColumn(HColumnProtos.HColumn value) { 841 | if (columnBuilder_ == null) { 842 | if (value == null) { 843 | throw new NullPointerException(); 844 | } 845 | ensureColumnIsMutable(); 846 | column_.add(value); 847 | onChanged(); 848 | } else { 849 | columnBuilder_.addMessage(value); 850 | } 851 | return this; 852 | } 853 | /** 854 | * repeated .HColumn column = 3; 855 | */ 856 | public Builder addColumn( 857 | int index, HColumnProtos.HColumn value) { 858 | if (columnBuilder_ == null) { 859 | if (value == null) { 860 | throw new NullPointerException(); 861 | } 862 | ensureColumnIsMutable(); 863 | column_.add(index, value); 864 | onChanged(); 865 | } else { 866 | columnBuilder_.addMessage(index, value); 867 | } 868 | return this; 869 | } 870 | /** 871 | * repeated .HColumn column = 3; 872 | */ 873 | public Builder addColumn( 874 | HColumnProtos.HColumn.Builder builderForValue) { 875 | if (columnBuilder_ == null) { 876 | ensureColumnIsMutable(); 877 | column_.add(builderForValue.build()); 878 | onChanged(); 879 | } else { 880 | columnBuilder_.addMessage(builderForValue.build()); 881 | } 882 | return this; 883 | } 884 | /** 885 | * repeated .HColumn column = 3; 886 | */ 887 | public Builder addColumn( 888 | int index, HColumnProtos.HColumn.Builder builderForValue) { 889 | if (columnBuilder_ == null) { 890 | ensureColumnIsMutable(); 891 | column_.add(index, builderForValue.build()); 892 | onChanged(); 893 | } else { 894 | columnBuilder_.addMessage(index, builderForValue.build()); 895 | } 896 | return this; 897 | } 898 | /** 899 | * repeated .HColumn column = 3; 900 | */ 901 | public Builder addAllColumn( 902 | Iterable values) { 903 | if (columnBuilder_ == null) { 904 | ensureColumnIsMutable(); 905 | super.addAll(values, column_); 906 | onChanged(); 907 | } else { 908 | columnBuilder_.addAllMessages(values); 909 | } 910 | return this; 911 | } 912 | /** 913 | * repeated .HColumn column = 3; 914 | */ 915 | public Builder clearColumn() { 916 | if (columnBuilder_ == null) { 917 | column_ = java.util.Collections.emptyList(); 918 | bitField0_ = (bitField0_ & ~0x00000004); 919 | onChanged(); 920 | } else { 921 | columnBuilder_.clear(); 922 | } 923 | return this; 924 | } 925 | /** 926 | * repeated .HColumn column = 3; 927 | */ 928 | public Builder removeColumn(int index) { 929 | if (columnBuilder_ == null) { 930 | ensureColumnIsMutable(); 931 | column_.remove(index); 932 | onChanged(); 933 | } else { 934 | columnBuilder_.remove(index); 935 | } 936 | return this; 937 | } 938 | /** 939 | * repeated .HColumn column = 3; 940 | */ 941 | public HColumnProtos.HColumn.Builder getColumnBuilder( 942 | int index) { 943 | return getColumnFieldBuilder().getBuilder(index); 944 | } 945 | /** 946 | * repeated .HColumn column = 3; 947 | */ 948 | public HColumnProtos.HColumnOrBuilder getColumnOrBuilder( 949 | int index) { 950 | if (columnBuilder_ == null) { 951 | return column_.get(index); } else { 952 | return columnBuilder_.getMessageOrBuilder(index); 953 | } 954 | } 955 | /** 956 | * repeated .HColumn column = 3; 957 | */ 958 | public java.util.List 959 | getColumnOrBuilderList() { 960 | if (columnBuilder_ != null) { 961 | return columnBuilder_.getMessageOrBuilderList(); 962 | } else { 963 | return java.util.Collections.unmodifiableList(column_); 964 | } 965 | } 966 | /** 967 | * repeated .HColumn column = 3; 968 | */ 969 | public HColumnProtos.HColumn.Builder addColumnBuilder() { 970 | return getColumnFieldBuilder().addBuilder( 971 | HColumnProtos.HColumn.getDefaultInstance()); 972 | } 973 | /** 974 | * repeated .HColumn column = 3; 975 | */ 976 | public HColumnProtos.HColumn.Builder addColumnBuilder( 977 | int index) { 978 | return getColumnFieldBuilder().addBuilder( 979 | index, HColumnProtos.HColumn.getDefaultInstance()); 980 | } 981 | /** 982 | * repeated .HColumn column = 3; 983 | */ 984 | public java.util.List 985 | getColumnBuilderList() { 986 | return getColumnFieldBuilder().getBuilderList(); 987 | } 988 | private com.google.protobuf.RepeatedFieldBuilder< 989 | HColumnProtos.HColumn, HColumnProtos.HColumn.Builder, HColumnProtos.HColumnOrBuilder> 990 | getColumnFieldBuilder() { 991 | if (columnBuilder_ == null) { 992 | columnBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< 993 | HColumnProtos.HColumn, HColumnProtos.HColumn.Builder, HColumnProtos.HColumnOrBuilder>( 994 | column_, 995 | ((bitField0_ & 0x00000004) == 0x00000004), 996 | getParentForChildren(), 997 | isClean()); 998 | column_ = null; 999 | } 1000 | return columnBuilder_; 1001 | } 1002 | 1003 | // @@protoc_insertion_point(builder_scope:HRow) 1004 | } 1005 | 1006 | static { 1007 | defaultInstance = new Row(true); 1008 | defaultInstance.initFields(); 1009 | } 1010 | 1011 | // @@protoc_insertion_point(class_scope:HRow) 1012 | } 1013 | 1014 | private static com.google.protobuf.Descriptors.Descriptor 1015 | internal_static_Row_descriptor; 1016 | private static 1017 | com.google.protobuf.GeneratedMessage.FieldAccessorTable 1018 | internal_static_Row_fieldAccessorTable; 1019 | 1020 | public static com.google.protobuf.Descriptors.FileDescriptor 1021 | getDescriptor() { 1022 | return descriptor; 1023 | } 1024 | private static com.google.protobuf.Descriptors.FileDescriptor 1025 | descriptor; 1026 | static { 1027 | String[] descriptorData = { 1028 | "\n\nHRow.proto\032\rHColumn.proto\"@\n\003Row\022\013\n\003ro" + 1029 | "w\030\001 \001(\014\022\022\n\002op\030\002 \001(\0162\006.RowOp\022\030\n\006column\030\003 " + 1030 | "\003(\0132\010.HColumn*\034\n\005RowOp\022\007\n\003PUT\020\001\022\n\n\006DELET" + 1031 | "E\020\002B6\n#com.sv.hbase.cdc.protobuf.generat" + 1032 | "edB\nHRowProtosH\001\240\001\001" 1033 | }; 1034 | com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = 1035 | new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { 1036 | public com.google.protobuf.ExtensionRegistry assignDescriptors( 1037 | com.google.protobuf.Descriptors.FileDescriptor root) { 1038 | descriptor = root; 1039 | internal_static_Row_descriptor = 1040 | getDescriptor().getMessageTypes().get(0); 1041 | internal_static_Row_fieldAccessorTable = new 1042 | com.google.protobuf.GeneratedMessage.FieldAccessorTable( 1043 | internal_static_Row_descriptor, 1044 | new String[] { "HRow", "Op", "HColumn", }); 1045 | return null; 1046 | } 1047 | }; 1048 | com.google.protobuf.Descriptors.FileDescriptor 1049 | .internalBuildGeneratedFileFrom(descriptorData, 1050 | new com.google.protobuf.Descriptors.FileDescriptor[] { 1051 | HColumnProtos.getDescriptor(), 1052 | }, assigner); 1053 | } 1054 | 1055 | // @@protoc_insertion_point(outer_class_scope) 1056 | } 1057 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/serde/HRowProtobufSerde.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc.serde; 19 | 20 | import com.google.protobuf.InvalidProtocolBufferException; 21 | import io.svectors.hbase.cdc.ReplicationException; 22 | import io.svectors.hbase.cdc.model.HRow; 23 | import io.svectors.hbase.cdc.protobuf.generated.HColumnProtos; 24 | import io.svectors.hbase.cdc.protobuf.generated.HRowProtos; 25 | import org.apache.hadoop.hbase.util.ByteStringer; 26 | import org.apache.kafka.common.serialization.Deserializer; 27 | import org.apache.kafka.common.serialization.Serializer; 28 | 29 | import java.util.List; 30 | import java.util.Map; 31 | import java.util.stream.Collectors; 32 | 33 | /** 34 | * @author ravi.magham 35 | */ 36 | public class HRowProtobufSerde implements Serializer, Deserializer { 37 | 38 | @Override 39 | public void configure(Map configs, boolean isKey) { 40 | // NO-OP 41 | } 42 | 43 | @Override 44 | public HRow deserialize(String topic, byte[] data) { 45 | try { 46 | HRowProtos.Row rowProto = HRowProtos.Row.parseFrom(data); 47 | List columnsProto = rowProto.getColumnList(); 48 | 49 | List columns = columnsProto.stream().map(col -> { 50 | HRow.HColumn column = new HRow.HColumn(col.getFamily().toByteArray(), 51 | col.getQualifier().toByteArray(), 52 | col.getValue().toByteArray(), 53 | col.getTimestamp()); 54 | return column; 55 | 56 | }).collect(Collectors.toList()); 57 | final byte[] rowkey = rowProto.getRow().toByteArray(); 58 | final HRow.RowOp rowOp = HRow.RowOp.valueOf(rowProto.getOp().name()); 59 | final HRow row = new HRow(rowkey, rowOp, columns); 60 | return row; 61 | } catch (InvalidProtocolBufferException e) { 62 | throw new ReplicationException(e); 63 | } 64 | } 65 | 66 | /** 67 | * Serializes each {@link HRow} to byte[] through protobuf. 68 | * @param topic 69 | * @param row 70 | * @return 71 | */ 72 | @Override 73 | public byte[] serialize(String topic, HRow row) { 74 | final HRowProtos.Row.Builder rowBuilder = HRowProtos.Row.newBuilder(); 75 | row.getColumns().stream().forEach(column -> { 76 | HColumnProtos.HColumn.Builder cellBuilder = HColumnProtos.HColumn.newBuilder(); 77 | cellBuilder.setFamily(ByteStringer.wrap(column.getFamily())); 78 | cellBuilder.setQualifier(ByteStringer.wrap(column.getQualifier())); 79 | cellBuilder.setValue(ByteStringer.wrap(column.getValue())); 80 | cellBuilder.setTimestamp(column.getTimestamp()); 81 | rowBuilder.addColumn(cellBuilder.build()); 82 | }); 83 | rowBuilder.setOp(HRowProtos.RowOp.valueOf(row.getRowOp().name())); 84 | rowBuilder.setRow(ByteStringer.wrap(row.getRowKey())); 85 | return rowBuilder.build().toByteArray(); 86 | } 87 | 88 | 89 | @Override 90 | public void close() { 91 | // NO-OP 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /src/main/java/io/svectors/hbase/cdc/util/TopicNameFilter.java: -------------------------------------------------------------------------------- 1 | package io.svectors.hbase.cdc.util; 2 | 3 | import com.google.common.collect.Sets; 4 | import io.svectors.hbase.cdc.config.KafkaConfiguration; 5 | 6 | import java.util.Optional; 7 | import java.util.Set; 8 | import java.util.function.Predicate; 9 | 10 | /** 11 | * @author ravi.magham 12 | */ 13 | public class TopicNameFilter implements Predicate { 14 | 15 | private Set validTopics = Sets.newHashSet(); 16 | 17 | public TopicNameFilter(final KafkaConfiguration configuration) { 18 | final Optional topics = configuration.getWhitelistTopics(); 19 | if(topics.isPresent()) { 20 | final String[] whitelistTopics = topics.toString().split(","); 21 | for(String topic : whitelistTopics) { 22 | validTopics.add(topic); 23 | } 24 | } 25 | } 26 | 27 | /** 28 | * Checks if the topic name passed is a whitelist. 29 | * @param topic 30 | * @return 31 | */ 32 | @Override 33 | public boolean test(String topic) { 34 | return validTopics.isEmpty() 35 | || validTopics.contains(topic); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/resources/protobuf/HColumn.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | // HColumn 20 | option java_package = "io.supportvectors.hbase.cdc.protobuf.generated"; 21 | option java_outer_classname = "HColumnProtos"; 22 | option java_generate_equals_and_hash = true; 23 | option optimize_for = SPEED; 24 | 25 | /** 26 | * Protocol buffer of each HColumn. 27 | */ 28 | message HColumn { 29 | optional bytes family = 1; 30 | optional bytes qualifier = 2; 31 | optional uint64 timestamp = 3; 32 | optional bytes value = 4; 33 | } 34 | -------------------------------------------------------------------------------- /src/main/resources/protobuf/HRow.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | // proto for HRow. 20 | 21 | option java_package = "io.supportvectors.hbase.cdc.protobuf.generated"; 22 | option java_outer_classname = "HRowProtos"; 23 | option java_generate_equals_and_hash = true; 24 | option optimize_for = SPEED; 25 | 26 | import "HColumn.proto"; 27 | 28 | /** 29 | * The row operation . for now , it is either Put or delete. 30 | */ 31 | enum RowOp { 32 | PUT = 1; 33 | DELETE = 2; 34 | } 35 | 36 | /** 37 | * Protocol buffer of HRow. 38 | */ 39 | message Row { 40 | optional bytes row = 1; 41 | optional RowOp op = 2; 42 | repeated HColumn column = 3; 43 | 44 | } 45 | -------------------------------------------------------------------------------- /src/test/java/io/svectors/hbase/cdc/BaseTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import org.apache.hadoop.conf.Configuration; 21 | import org.apache.hadoop.hbase.HBaseConfiguration; 22 | import org.apache.hadoop.hbase.HBaseTestingUtility; 23 | import org.apache.hadoop.hbase.HConstants; 24 | import org.apache.hadoop.hbase.TableName; 25 | import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; 26 | import org.apache.hadoop.hbase.zookeeper.ZKConfig; 27 | import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; 28 | import org.apache.hadoop.hbase.replication.ReplicationException; 29 | import org.apache.kafka.clients.consumer.ConsumerConfig; 30 | import org.apache.kafka.clients.consumer.KafkaConsumer; 31 | import org.junit.After; 32 | import org.junit.Before; 33 | 34 | import io.svectors.hbase.cdc.model.HRow; 35 | 36 | import java.io.IOException; 37 | import java.util.List; 38 | import java.util.Map; 39 | import java.util.Properties; 40 | 41 | /** 42 | * @author ravi.magham 43 | */ 44 | public abstract class BaseTest { 45 | 46 | protected HBaseTestingUtility utility; 47 | protected int numRegionServers; 48 | protected KafkaServer kafkaServer; 49 | 50 | @Before 51 | public void setUp() throws Exception { 52 | final Configuration hbaseConf = HBaseConfiguration.create(); 53 | hbaseConf.setInt("replication.stats.thread.period.seconds", 5); 54 | hbaseConf.setLong("replication.sleep.before.failover", 2000); 55 | hbaseConf.setInt("replication.source.maxretriesmultiplier", 10); 56 | hbaseConf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true); 57 | 58 | // add kafka properties. we prefix each property with kafka 59 | addKafkaProperties(hbaseConf); 60 | 61 | utility = new HBaseTestingUtility(hbaseConf); 62 | utility.startMiniCluster(); 63 | numRegionServers = utility.getHBaseCluster().getRegionServerThreads().size(); 64 | 65 | // setup kafka 66 | kafkaServer = new KafkaServer(utility.getZkCluster().getClientPort(), 9092); 67 | 68 | } 69 | 70 | /** 71 | * Add kafka properties to {@link Configuration} 72 | * @param hbaseConf 73 | */ 74 | private void addKafkaProperties(Configuration hbaseConf) { 75 | hbaseConf.set("kafka.bootstrap.servers", "localhost:9092"); 76 | hbaseConf.set("kafka.acks", "1"); 77 | hbaseConf.set("kafka.producer.type", "async"); 78 | hbaseConf.set("kafka.key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); 79 | hbaseConf.set("kafka.value.serializer", "io.svectors.hbase.cdc.serde.HRowProtobufSerde"); 80 | } 81 | 82 | /** 83 | * 84 | * @param configuration 85 | * @param peerName 86 | * @param tableCFs 87 | * @throws ReplicationException 88 | * @throws IOException 89 | */ 90 | protected void addPeer(final Configuration configuration,String peerName, Map> tableCFs) 91 | throws ReplicationException, IOException { 92 | try (ReplicationAdmin replicationAdmin = new ReplicationAdmin(configuration)) { 93 | ReplicationPeerConfig peerConfig = new ReplicationPeerConfig() 94 | .setClusterKey(ZKConfig.getZooKeeperClusterKey(configuration)) 95 | .setReplicationEndpointImpl(HbaseEndpoint.class.getName()); 96 | 97 | replicationAdmin.addPeer(peerName, peerConfig, tableCFs); 98 | } 99 | } 100 | 101 | @After 102 | public void tearDown() throws Exception { 103 | if(kafkaServer != null) { 104 | kafkaServer.shutdown(); 105 | } 106 | if(utility != null) { 107 | utility.shutdownMiniCluster(); 108 | } 109 | } 110 | 111 | /** 112 | * Creates and get the Kafka consumer 113 | * @return 114 | */ 115 | public KafkaConsumer createAndGetKafkaConsumer() { 116 | final Properties props = new Properties(); 117 | props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaServer.getBrokerConnectionString()); 118 | props.put("zookeeper.connect", kafkaServer.getZookeeperQuorum()); 119 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "testing"); 120 | props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); 121 | props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "1000"); 122 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); 123 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "io.svectors.hbase.cdc.serde.HRowProtobufSerde"); 124 | props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); 125 | final KafkaConsumer consumer = new KafkaConsumer<>(props); 126 | return consumer; 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /src/test/java/io/svectors/hbase/cdc/KafkaServer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import com.google.common.base.Preconditions; 21 | import kafka.server.KafkaConfig; 22 | import kafka.server.KafkaServerStartable; 23 | import java.io.File; 24 | import java.io.IOException; 25 | import java.util.Properties; 26 | import java.util.function.Function; 27 | 28 | /** 29 | * @author ravi.magham 30 | */ 31 | public class KafkaServer { 32 | 33 | private final int brokerPort; // kafka broker port 34 | private final int zookeeperPort; // zookeeper port 35 | private final KafkaServerStartable kafka; 36 | private final File logDir; 37 | 38 | private final Function TO_LOCAL_URI = (port) -> "localhost:"+ port; 39 | 40 | public KafkaServer(int zookeeperPort, int kafkaBrokerPort) { 41 | try { 42 | Preconditions.checkArgument(zookeeperPort > 0); 43 | Preconditions.checkArgument(kafkaBrokerPort > 0); 44 | this.zookeeperPort = zookeeperPort; 45 | this.brokerPort = kafkaBrokerPort; 46 | this.logDir = new File(System.getProperty("java.io.tmpdir"), "kafka/logs/hbase-cdc-kafka-" + brokerPort); 47 | 48 | KafkaConfig config = buildKafkaConfig(zookeeperPort); 49 | kafka = new KafkaServerStartable(config); 50 | kafka.startup(); 51 | } catch (Exception ex) { 52 | throw new RuntimeException("Could not start test broker", ex); 53 | } 54 | } 55 | 56 | private kafka.server.KafkaConfig buildKafkaConfig(int zookeeperPort) throws IOException { 57 | Properties p = new Properties(); 58 | p.setProperty("zookeeper.connect", TO_LOCAL_URI.apply(zookeeperPort)); 59 | p.setProperty("broker.id", "0"); 60 | p.setProperty("port", Integer.toString(this.brokerPort)); 61 | p.setProperty("log.dirs", logDir.getAbsolutePath()); 62 | p.put("host.name", "localhost"); 63 | p.put("controlled.shutdown.enable", "true"); 64 | 65 | return new KafkaConfig(p); 66 | } 67 | 68 | public String getBrokerConnectionString() { 69 | return TO_LOCAL_URI.apply(this.brokerPort); 70 | } 71 | 72 | protected String getZookeeperQuorum() { 73 | return TO_LOCAL_URI.apply(zookeeperPort); 74 | } 75 | 76 | public void shutdown() { 77 | kafka.shutdown(); 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /src/test/java/io/svectors/hbase/cdc/TestKafkaReplication.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package io.svectors.hbase.cdc; 19 | 20 | import static org.apache.hadoop.hbase.util.Bytes.toBytes; 21 | 22 | import io.svectors.hbase.cdc.model.HRow; 23 | import org.apache.hadoop.hbase.*; 24 | import org.apache.hadoop.hbase.client.ConnectionFactory; 25 | import org.apache.hadoop.hbase.client.HBaseAdmin; 26 | import org.apache.hadoop.hbase.client.Put; 27 | import org.apache.hadoop.hbase.client.Table; 28 | import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; 29 | import org.apache.hadoop.hbase.replication.ReplicationException; 30 | import org.apache.hadoop.hbase.util.Bytes; 31 | import org.apache.kafka.clients.consumer.ConsumerRecords; 32 | import org.apache.kafka.clients.consumer.KafkaConsumer; 33 | import org.junit.Assert; 34 | import org.junit.Test; 35 | import org.slf4j.Logger; 36 | import org.slf4j.LoggerFactory; 37 | import java.io.IOException; 38 | import java.util.ArrayList; 39 | import java.util.Collections; 40 | import java.util.HashMap; 41 | import java.util.List; 42 | import java.util.Map; 43 | import java.util.concurrent.atomic.AtomicInteger; 44 | 45 | /** 46 | * @author ravi.magham 47 | */ 48 | public class TestKafkaReplication extends BaseTest { 49 | 50 | public static final Logger LOG = LoggerFactory.getLogger(TestKafkaReplication.class); 51 | 52 | private static final String PEER_NAME = "hbase.cdc.kafka"; 53 | protected final TableName TABLE_NAME = TableName.valueOf("testings"); 54 | protected final String ROWKEY = "rk-%s"; 55 | protected final String COLUMN_FAMILY = "d"; 56 | protected final String QUALIFIER = "q"; 57 | protected final String VALUE = "v"; 58 | 59 | @Test 60 | public void testCustomReplicationEndpoint() throws Exception { 61 | try { 62 | Map> tableCfs = new HashMap<>(); 63 | List cfs = new ArrayList<>(); 64 | cfs.add(COLUMN_FAMILY); 65 | tableCfs.put(TABLE_NAME, cfs); 66 | 67 | createTestTable(); 68 | addPeer(utility.getConfiguration(), PEER_NAME, tableCfs); 69 | int numberOfRecords = 10; 70 | addData(numberOfRecords); 71 | 72 | final KafkaConsumer kafkaConsumer = createAndGetKafkaConsumer(); 73 | final AtomicInteger totalRecords = new AtomicInteger(0); 74 | kafkaConsumer.subscribe(Collections.singletonList(TABLE_NAME.getNameAsString())); 75 | while (totalRecords.get() < numberOfRecords) { 76 | ConsumerRecords consumerRecords = kafkaConsumer.poll(1000); 77 | if(consumerRecords != null && !consumerRecords.isEmpty()) { 78 | consumerRecords.forEach(record -> { 79 | final String expectedRowkey = String.format(ROWKEY, totalRecords.getAndAdd(1)); 80 | Assert.assertEquals(expectedRowkey, Bytes.toString(record.value().getRowKey())); 81 | }); 82 | } 83 | } 84 | kafkaConsumer.close(); 85 | } finally { 86 | removePeer(); 87 | } 88 | } 89 | 90 | /** 91 | * Create the hbase table with a scope set to Global 92 | * @throws IOException 93 | */ 94 | private void createTestTable() throws IOException { 95 | try(HBaseAdmin hBaseAdmin = utility.getHBaseAdmin()) { 96 | final HTableDescriptor hTableDescriptor = new HTableDescriptor(TABLE_NAME); 97 | final HColumnDescriptor hColumnDescriptor = new HColumnDescriptor(COLUMN_FAMILY); 98 | hColumnDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); 99 | hTableDescriptor.addFamily(hColumnDescriptor); 100 | hBaseAdmin.createTable(hTableDescriptor); 101 | } 102 | utility.waitUntilAllRegionsAssigned(TABLE_NAME); 103 | } 104 | 105 | /** 106 | * Adds data to the previously created HBase table 107 | * @throws IOException 108 | */ 109 | private void addData(int numberOfRecords) throws IOException { 110 | try(Table hTable = ConnectionFactory.createConnection(utility.getConfiguration()).getTable(TABLE_NAME)) { 111 | for(int i = 0; i < numberOfRecords; i++) { 112 | Put put = new Put(toBytes(String.format(ROWKEY, i))); 113 | put.addColumn(toBytes(COLUMN_FAMILY), toBytes(QUALIFIER), toBytes(VALUE)); 114 | hTable.put(put); 115 | } 116 | } 117 | } 118 | 119 | /** 120 | * Removes the peer 121 | * @throws IOException 122 | * @throws ReplicationException 123 | */ 124 | private void removePeer() throws IOException, ReplicationException { 125 | try(ReplicationAdmin replicationAdmin = new ReplicationAdmin(utility.getConfiguration())) { 126 | replicationAdmin.removePeer(PEER_NAME); 127 | } 128 | } 129 | } 130 | --------------------------------------------------------------------------------