AHeise commented on a change in pull request #15109: URL: https://github.com/apache/flink/pull/15109#discussion_r594607784
########## File path: flink-connectors/flink-connector-hbase/README.md ########## @@ -0,0 +1,118 @@ +# Flink HBase Connector + +This module provides connectors that allow Flink to access [HBase](https://hbase.apache.org/) using [CDC](https://en.wikipedia.org/wiki/Change_data_capture). +It supports the new Source and Sink API specified in [FLIP-27](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface) and [FLIP-143](https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API). + +## Installing HBase + +Follow the instructions from the [HBase Quick Start Guide](http://hbase.apache.org/book.html#quickstart) to install HBase. + +*Version Compatibility*: This module is compatible with Apache HBase *2.3.4*. Review comment: Can you rephrase that it's working with 2.1.0+ (to be clarified) + you tested with 2.3.4. As it stands, it's not terribly useful for existing setups. ########## File path: flink-connectors/flink-connector-hbase/README.md ########## @@ -0,0 +1,118 @@ +# Flink HBase Connector + +This module provides connectors that allow Flink to access [HBase](https://hbase.apache.org/) using [CDC](https://en.wikipedia.org/wiki/Change_data_capture). +It supports the new Source and Sink API specified in [FLIP-27](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface) and [FLIP-143](https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API). + +## Installing HBase + +Follow the instructions from the [HBase Quick Start Guide](http://hbase.apache.org/book.html#quickstart) to install HBase. + +*Version Compatibility*: This module is compatible with Apache HBase *2.3.4*. + +## HBase Configuration + +Connecting to HBase always requires a `Configuration` instance. +If there is an HBase gateway on the same host as the Flink gateway where the application is started, this can be obtained by invoking `HBaseConfiguration.create()` as in the examples below. +If that's not the case a configuration should be provided where the proper core-site, hdfs-site, and hbase-site are added as resources. + +## DataStream API + +### Reading data from HBase + +To receive data from HBase, the connector makes use of the internal replication mechanism of HBase. +The connector registers at the HBase cluster as a *Replication Peer* and will receive all change events from HBase. + +For the replication to work, the HBase config needs to have replication enabled in the `hbase-site.xml` file. +This needs be done only once per cluster: +```xml +<configuration> + <property> + <name>hbase.replication</name> + <value>true</value> + </property> + ... +</configuration> +``` +All incoming events to Flink will be processed as an `HBaseSourceEvent`. +You will need to specify a Deserializer which will transform each event from an `HBaseSourceEvent` to the desired DataStream type. + +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +Configuration hbaseConfig = HBaseConfiguration.create(); +String tableName = "TestTable"; + +HBaseSource<String> hbaseSource = + HBaseSource.<String>builder() Review comment: Make builder untyped (`<?>`) until deserializer set? ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/hbaseendpoint/HBaseEndpoint.java ########## @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.source.hbaseendpoint; + +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; +import org.apache.flink.connector.hbase.source.reader.HBaseSourceEvent; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; + +/** Consumer of HBase WAL edits. */ +public class HBaseEndpoint implements ReplicationTargetInterface { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseEndpoint.class); + private final String clusterKey; + /** The id under which the replication target is made known to the source cluster. */ + private final String replicationPeerId; + + private final Configuration hbaseConf; + private final RecoverableZooKeeper zooKeeper; + private final RpcServer rpcServer; + private final FutureCompletingBlockingQueue<HBaseSourceEvent> walEdits; + private final String hostName; + private final String tableName; + private boolean isRunning = false; + + public HBaseEndpoint(byte[] serializedConfig, Properties properties) + throws IOException, KeeperException, InterruptedException { + this(HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null), properties); + } + + public HBaseEndpoint(Configuration hbaseConf, Properties properties) + throws InterruptedException, KeeperException, IOException { + this(UUID.randomUUID().toString().substring(0, 5), hbaseConf, properties); + } + + public HBaseEndpoint(String peerId, Configuration hbaseConf, Properties properties) + throws IOException, KeeperException, InterruptedException { + this.hbaseConf = hbaseConf; + this.clusterKey = peerId + "_clusterKey"; + this.replicationPeerId = peerId; + this.hostName = HBaseSourceOptions.getHostName(properties); + this.tableName = HBaseSourceOptions.getTableName(properties); + int queueCapacity = HBaseSourceOptions.getEndpointQueueCapacity(properties); + this.walEdits = new FutureCompletingBlockingQueue<>(queueCapacity); + + // Setup + zooKeeper = connectToZooKeeper(); + rpcServer = createServer(); + registerAtZooKeeper(); + } + + private RecoverableZooKeeper connectToZooKeeper() throws IOException { + // Using private ZKUtil and RecoverableZooKeeper here, because the stability is so much + // improved + RecoverableZooKeeper zooKeeper = + ZKUtil.connect(hbaseConf, getZookeeperClientAddress(), null); + LOG.debug("Connected to Zookeeper"); + return zooKeeper; + } + + private RpcServer createServer() throws IOException { + Server server = new EmptyHBaseServer(hbaseConf); + InetSocketAddress initialIsa = new InetSocketAddress(hostName, 0); + String name = "regionserver/" + initialIsa.toString(); + + RpcServer.BlockingServiceAndInterface bsai = + new RpcServer.BlockingServiceAndInterface( + AdminProtos.AdminService.newReflectiveBlockingService(this), + BlockingInterface.class); + RpcServer rpcServer = + RpcServerFactory.createRpcServer( + server, + name, + Arrays.asList(bsai), + initialIsa, + hbaseConf, + new FifoRpcScheduler( + hbaseConf, + hbaseConf.getInt("hbase.regionserver.handler.count", 10))); + + rpcServer.start(); + LOG.debug("Started rpc server at {}", initialIsa); + return rpcServer; + } + + private void registerAtZooKeeper() throws KeeperException, InterruptedException { + createZKPath(getBaseString() + "/" + clusterKey, null, CreateMode.PERSISTENT); + createZKPath(getBaseString() + "/" + clusterKey + "/rs", null, CreateMode.PERSISTENT); + + UUID uuid = UUID.nameUUIDFromBytes(Bytes.toBytes(clusterKey)); + createZKPath( + getBaseString() + "/" + clusterKey + "/hbaseid", + Bytes.toBytes(uuid.toString()), + CreateMode.PERSISTENT); + + ServerName serverName = + ServerName.valueOf( + hostName, + rpcServer.getListenerAddress().getPort(), + System.currentTimeMillis()); + createZKPath( + getBaseString() + "/" + clusterKey + "/rs/" + serverName.getServerName(), + null, + CreateMode.EPHEMERAL); + + LOG.debug("Registered rpc server node at zookeeper"); + } + + /** + * Blocks as long as the queue is empty. If the queue isn't empty it will try and get as many + * elements as possible from the queue. It is not guaranteed that at least one element is in the + * returned list. + * + * @return a list of {@link HBaseSourceEvent}. + */ + public List<HBaseSourceEvent> getAll() { + if (!isRunning) { + // Protects from infinite waiting + throw new RuntimeException("Consumer is not running"); + } + List<HBaseSourceEvent> elements = new ArrayList<>(); + HBaseSourceEvent event; + + try { + walEdits.getAvailabilityFuture().get(); + while ((event = walEdits.poll()) != null) { + elements.add(event); + } + return elements; + } catch (InterruptedException | ExecutionException e) { Review comment: 👍 ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/hbaseendpoint/HBaseEndpoint.java ########## @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.source.hbaseendpoint; + +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; +import org.apache.flink.connector.hbase.source.reader.HBaseSourceEvent; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; + +/** Consumer of HBase WAL edits. */ +public class HBaseEndpoint implements ReplicationTargetInterface { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseEndpoint.class); + private final String clusterKey; + /** The id under which the replication target is made known to the source cluster. */ + private final String replicationPeerId; + + private final Configuration hbaseConf; + private final RecoverableZooKeeper zooKeeper; + private final RpcServer rpcServer; + private final FutureCompletingBlockingQueue<HBaseSourceEvent> walEdits; + private final String hostName; + private final String tableName; + private boolean isRunning = false; + + public HBaseEndpoint(byte[] serializedConfig, Properties properties) + throws IOException, KeeperException, InterruptedException { + this(HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null), properties); + } + + public HBaseEndpoint(Configuration hbaseConf, Properties properties) + throws InterruptedException, KeeperException, IOException { + this(UUID.randomUUID().toString().substring(0, 5), hbaseConf, properties); + } + + public HBaseEndpoint(String peerId, Configuration hbaseConf, Properties properties) + throws IOException, KeeperException, InterruptedException { + this.hbaseConf = hbaseConf; + this.clusterKey = peerId + "_clusterKey"; + this.replicationPeerId = peerId; + this.hostName = HBaseSourceOptions.getHostName(properties); + this.tableName = HBaseSourceOptions.getTableName(properties); + int queueCapacity = HBaseSourceOptions.getEndpointQueueCapacity(properties); + this.walEdits = new FutureCompletingBlockingQueue<>(queueCapacity); + + // Setup + zooKeeper = connectToZooKeeper(); + rpcServer = createServer(); + registerAtZooKeeper(); + } + + private RecoverableZooKeeper connectToZooKeeper() throws IOException { + // Using private ZKUtil and RecoverableZooKeeper here, because the stability is so much + // improved + RecoverableZooKeeper zooKeeper = + ZKUtil.connect(hbaseConf, getZookeeperClientAddress(), null); + LOG.debug("Connected to Zookeeper"); + return zooKeeper; + } + + private RpcServer createServer() throws IOException { + Server server = new EmptyHBaseServer(hbaseConf); + InetSocketAddress initialIsa = new InetSocketAddress(hostName, 0); + String name = "regionserver/" + initialIsa.toString(); + + RpcServer.BlockingServiceAndInterface bsai = + new RpcServer.BlockingServiceAndInterface( + AdminProtos.AdminService.newReflectiveBlockingService(this), + BlockingInterface.class); + RpcServer rpcServer = + RpcServerFactory.createRpcServer( + server, + name, + Arrays.asList(bsai), + initialIsa, + hbaseConf, + new FifoRpcScheduler( + hbaseConf, + hbaseConf.getInt("hbase.regionserver.handler.count", 10))); + + rpcServer.start(); + LOG.debug("Started rpc server at {}", initialIsa); + return rpcServer; + } + + private void registerAtZooKeeper() throws KeeperException, InterruptedException { + createZKPath(getBaseString() + "/" + clusterKey, null, CreateMode.PERSISTENT); + createZKPath(getBaseString() + "/" + clusterKey + "/rs", null, CreateMode.PERSISTENT); + + UUID uuid = UUID.nameUUIDFromBytes(Bytes.toBytes(clusterKey)); + createZKPath( + getBaseString() + "/" + clusterKey + "/hbaseid", + Bytes.toBytes(uuid.toString()), + CreateMode.PERSISTENT); + + ServerName serverName = + ServerName.valueOf( + hostName, + rpcServer.getListenerAddress().getPort(), + System.currentTimeMillis()); + createZKPath( + getBaseString() + "/" + clusterKey + "/rs/" + serverName.getServerName(), + null, + CreateMode.EPHEMERAL); + + LOG.debug("Registered rpc server node at zookeeper"); + } + + /** + * Blocks as long as the queue is empty. If the queue isn't empty it will try and get as many + * elements as possible from the queue. It is not guaranteed that at least one element is in the + * returned list. + * + * @return a list of {@link HBaseSourceEvent}. + */ + public List<HBaseSourceEvent> getAll() { + if (!isRunning) { + // Protects from infinite waiting + throw new RuntimeException("Consumer is not running"); + } + List<HBaseSourceEvent> elements = new ArrayList<>(); + HBaseSourceEvent event; + + try { + walEdits.getAvailabilityFuture().get(); + while ((event = walEdits.poll()) != null) { + elements.add(event); + } + return elements; + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Can't retrieve elements from queue", e); + } + } + + public void wakeup() { + walEdits.notifyAvailable(); + } + + public void close() throws InterruptedException { + // TODO unregister replication peer [and delete zookeeper paths] + isRunning = false; + try { + zooKeeper.close(); + LOG.debug("Closed connection to ZooKeeper"); + } finally { + rpcServer.stop(); Review comment: Should probably be in reverse order. ########## File path: flink-connectors/flink-connector-hbase/src/test/java/org/apache/flink/connector/hbase/testutil/HBaseTestCluster.java ########## @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.testutil; + +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hbase.thirdparty.com.google.common.io.Closer; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** Provides static access to a {@link MiniHBaseCluster} for testing. */ +public class HBaseTestCluster extends ExternalResource { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseTestCluster.class); + + public static final String COLUMN_FAMILY_BASE = "info"; + public static final String DEFAULT_COLUMN_FAMILY = COLUMN_FAMILY_BASE + 0; + public static final String QUALIFIER_BASE = "qualifier"; + public static final String DEFAULT_QUALIFIER = QUALIFIER_BASE + 0; + + private MiniHBaseCluster cluster; + private Configuration hbaseConf; + private String testFolder; + + public HBaseTestCluster() {} + + /* + * How to use it + */ + public static void main(String[] args) throws Exception { + HBaseTestCluster hbaseTestCluster = new HBaseTestCluster(); + hbaseTestCluster.startCluster(); + hbaseTestCluster.makeTable("tableName"); + // ... + hbaseTestCluster.shutdownCluster(); + } + + public void startCluster() throws IOException, InterruptedException, ExecutionException { + LOG.info("Starting HBase test cluster ..."); + testFolder = Files.createTempDirectory(null).toString(); + + // Fallback for windows users with space in user name, will not work if path contains space. + if (testFolder.contains(" ")) { + testFolder = "/flink-hbase-test-data/"; + } + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("tempusername")); + + hbaseConf = HBaseConfiguration.create(); + hbaseConf.setInt("replication.stats.thread.period.seconds", 5); + hbaseConf.setLong("replication.sleep.before.failover", 2000); + hbaseConf.setInt("replication.source.maxretriesmultiplier", 10); + hbaseConf.setBoolean("hbase.replication", true); + + System.setProperty(HBaseTestingUtility.BASE_TEST_DIRECTORY_KEY, testFolder); + + HBaseTestingUtility utility = new HBaseTestingUtility(hbaseConf); + LOG.info("Testfolder: {}", utility.getDataTestDir().toString()); + try { + cluster = + utility.startMiniCluster( + StartMiniClusterOption.builder().numRegionServers(3).build()); + int numRegionServers = utility.getHBaseCluster().getRegionServerThreads().size(); + LOG.info("Number of region servers: {}", numRegionServers); + LOG.info( + "ZooKeeper client address: {}:{}", + hbaseConf.get("hbase.zookeeper.quorum"), + hbaseConf.get("hbase.zookeeper.property.clientPort")); + LOG.info( + "Master port={}, web UI at port={}", + hbaseConf.get("hbase.master.port"), + hbaseConf.get("hbase.master.info.port")); + + cluster.waitForActiveAndReadyMaster(30 * 1000); + HBaseAdmin.available(hbaseConf); + LOG.info("HBase test cluster up and running ..."); + + } catch (Exception e) { + throw new RuntimeException("Could not start HBase test mini cluster", e); + } + + assert canConnectToCluster(); + } + + public void shutdownCluster() { + LOG.info("Shutting down HBase test cluster"); + try { + try (Closer closer = Closer.create()) { + closer.register(this::clearTables); + closer.register(this::clearReplicationPeers); + } + try { + // Closer is not able to call this method correctly, instead logs process dump + cluster.shutdown(); + } catch (IOException e) { + LOG.error("Error while shutting down HBase test cluster", e); + } + try (Closer closer = Closer.create()) { + closer.register(this::waitForShutDown); + closer.register(Paths.get(testFolder).toFile()::delete); + } + } catch (IOException e) { + throw new RuntimeException( + "Failed to shut down HBase test cluster. Future program state might be compromised.", + e); + } + LOG.info("HBase test cluster shut down"); + } + + public void waitForShutDown() { + try { + CompletableFuture.runAsync(cluster::waitUntilShutDown).get(240, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error("Interrupted while waiting for HBase test cluster to shut down", e); + } catch (ExecutionException e) { + e.printStackTrace(); + LOG.error("Exception while waiting for HBase test cluster to shut down", e); + } catch (TimeoutException e) { + e.printStackTrace(); + LOG.error("Waiting for HBase test cluster to shut down timed out", e); + } + } + + public boolean canConnectToCluster() throws InterruptedException, ExecutionException { + try { + return CompletableFuture.supplyAsync( + () -> { + try (Connection connection = + ConnectionFactory.createConnection(getConfig())) { + return true; + } catch (IOException e) { + LOG.error("Error trying to connect to cluster", e); + return false; + } + }) + .get(10, TimeUnit.SECONDS); + } catch (TimeoutException e) { + LOG.error("Trying to connect to HBase test cluster timed out", e); + return false; + } + } + + public void clearTables() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + for (TableDescriptor table : admin.listTableDescriptors()) { + admin.disableTable(table.getTableName()); + admin.deleteTable(table.getTableName()); + } + } catch (IOException e) { + throw new RuntimeException("Could not clear test cluster tables", e); + } + } + + public void clearReplicationPeers() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + StringBuilder logMessage = new StringBuilder("Clearing existing replication peers:"); + for (ReplicationPeerDescription desc : admin.listReplicationPeers()) { + logMessage.append("\n\t").append(desc.getPeerId()).append(" | ").append(desc); + admin.removeReplicationPeer(desc.getPeerId()); Review comment: Same here, if you close inside a loop, the first error will skip the following entries. ########## File path: flink-connectors/flink-connector-hbase/src/test/java/org/apache/flink/connector/hbase/testutil/HBaseTestCluster.java ########## @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.testutil; + +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hbase.thirdparty.com.google.common.io.Closer; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** Provides static access to a {@link MiniHBaseCluster} for testing. */ +public class HBaseTestCluster extends ExternalResource { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseTestCluster.class); + + public static final String COLUMN_FAMILY_BASE = "info"; + public static final String DEFAULT_COLUMN_FAMILY = COLUMN_FAMILY_BASE + 0; + public static final String QUALIFIER_BASE = "qualifier"; + public static final String DEFAULT_QUALIFIER = QUALIFIER_BASE + 0; + + private MiniHBaseCluster cluster; + private Configuration hbaseConf; + private String testFolder; + + public HBaseTestCluster() {} + + /* + * How to use it + */ + public static void main(String[] args) throws Exception { + HBaseTestCluster hbaseTestCluster = new HBaseTestCluster(); + hbaseTestCluster.startCluster(); + hbaseTestCluster.makeTable("tableName"); + // ... + hbaseTestCluster.shutdownCluster(); + } + + public void startCluster() throws IOException, InterruptedException, ExecutionException { + LOG.info("Starting HBase test cluster ..."); + testFolder = Files.createTempDirectory(null).toString(); + + // Fallback for windows users with space in user name, will not work if path contains space. + if (testFolder.contains(" ")) { + testFolder = "/flink-hbase-test-data/"; + } + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("tempusername")); + + hbaseConf = HBaseConfiguration.create(); + hbaseConf.setInt("replication.stats.thread.period.seconds", 5); + hbaseConf.setLong("replication.sleep.before.failover", 2000); + hbaseConf.setInt("replication.source.maxretriesmultiplier", 10); + hbaseConf.setBoolean("hbase.replication", true); + + System.setProperty(HBaseTestingUtility.BASE_TEST_DIRECTORY_KEY, testFolder); + + HBaseTestingUtility utility = new HBaseTestingUtility(hbaseConf); + LOG.info("Testfolder: {}", utility.getDataTestDir().toString()); + try { + cluster = + utility.startMiniCluster( + StartMiniClusterOption.builder().numRegionServers(3).build()); + int numRegionServers = utility.getHBaseCluster().getRegionServerThreads().size(); + LOG.info("Number of region servers: {}", numRegionServers); + LOG.info( + "ZooKeeper client address: {}:{}", + hbaseConf.get("hbase.zookeeper.quorum"), + hbaseConf.get("hbase.zookeeper.property.clientPort")); + LOG.info( + "Master port={}, web UI at port={}", + hbaseConf.get("hbase.master.port"), + hbaseConf.get("hbase.master.info.port")); + + cluster.waitForActiveAndReadyMaster(30 * 1000); + HBaseAdmin.available(hbaseConf); + LOG.info("HBase test cluster up and running ..."); + + } catch (Exception e) { + throw new RuntimeException("Could not start HBase test mini cluster", e); + } + + assert canConnectToCluster(); + } + + public void shutdownCluster() { + LOG.info("Shutting down HBase test cluster"); + try { + try (Closer closer = Closer.create()) { + closer.register(this::clearTables); + closer.register(this::clearReplicationPeers); + } + try { + // Closer is not able to call this method correctly, instead logs process dump + cluster.shutdown(); + } catch (IOException e) { + LOG.error("Error while shutting down HBase test cluster", e); + } + try (Closer closer = Closer.create()) { + closer.register(this::waitForShutDown); + closer.register(Paths.get(testFolder).toFile()::delete); + } + } catch (IOException e) { + throw new RuntimeException( + "Failed to shut down HBase test cluster. Future program state might be compromised.", + e); + } + LOG.info("HBase test cluster shut down"); + } + + public void waitForShutDown() { + try { + CompletableFuture.runAsync(cluster::waitUntilShutDown).get(240, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error("Interrupted while waiting for HBase test cluster to shut down", e); + } catch (ExecutionException e) { + e.printStackTrace(); + LOG.error("Exception while waiting for HBase test cluster to shut down", e); + } catch (TimeoutException e) { + e.printStackTrace(); + LOG.error("Waiting for HBase test cluster to shut down timed out", e); + } + } + + public boolean canConnectToCluster() throws InterruptedException, ExecutionException { + try { + return CompletableFuture.supplyAsync( + () -> { + try (Connection connection = + ConnectionFactory.createConnection(getConfig())) { + return true; + } catch (IOException e) { + LOG.error("Error trying to connect to cluster", e); + return false; + } + }) + .get(10, TimeUnit.SECONDS); + } catch (TimeoutException e) { + LOG.error("Trying to connect to HBase test cluster timed out", e); + return false; + } + } + + public void clearTables() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + for (TableDescriptor table : admin.listTableDescriptors()) { + admin.disableTable(table.getTableName()); + admin.deleteTable(table.getTableName()); + } + } catch (IOException e) { + throw new RuntimeException("Could not clear test cluster tables", e); + } + } + + public void clearReplicationPeers() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + StringBuilder logMessage = new StringBuilder("Clearing existing replication peers:"); + for (ReplicationPeerDescription desc : admin.listReplicationPeers()) { Review comment: Use `getReplicationPeers`? ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSink.java ########## @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink; + +import org.apache.flink.api.connector.sink.Committer; +import org.apache.flink.api.connector.sink.GlobalCommitter; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.hbase.sink.writer.HBaseWriter; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.Mutation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Properties; + +/** + * A Sink Connector for HBase. Please use an {@link HBaseSinkBuilder} to construct a {@link + * HBaseSink}. The following example shows how to create an HBaseSink that writes Long values to + * HBase. + * + * <pre>{@code + * HBaseSink<Long> hbaseSink = + * HBaseSink.<Long>builder() + * .setTableName(tableName) + * .setSinkSerializer(new HBaseLongSerializer()) + * .setHBaseConfiguration(hbaseConfig) + * .build(); + * }</pre> + * + * <p>Here is an example for the Serializer: + * + * <pre>{@code + * static class HBaseLongSerializer implements HBaseSinkSerializer<Long> { + * @Override + * public HBaseEvent serialize(Long event) { + * return HBaseEvent.putWith( // or deleteWith() + * event.toString(), // rowId + * "exampleColumnFamily", // column family + * "exampleQualifier", // qualifier + * Bytes.toBytes(event.toString())); // payload + * } + * } + * }</pre> + * + * <p>See {@link HBaseSinkBuilder} for more details. Review comment: ``` @see HBaseSinkBuilder HBaseSinkBuilder for more details on creation. @see HBaseSinkSerializer HBaseSinkSerializer for more details on serialization. ``` ########## File path: flink-connectors/flink-connector-hbase/README.md ########## @@ -0,0 +1,118 @@ +# Flink HBase Connector + +This module provides connectors that allow Flink to access [HBase](https://hbase.apache.org/) using [CDC](https://en.wikipedia.org/wiki/Change_data_capture). +It supports the new Source and Sink API specified in [FLIP-27](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface) and [FLIP-143](https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API). + +## Installing HBase + +Follow the instructions from the [HBase Quick Start Guide](http://hbase.apache.org/book.html#quickstart) to install HBase. + +*Version Compatibility*: This module is compatible with Apache HBase *2.3.4*. + +## HBase Configuration + +Connecting to HBase always requires a `Configuration` instance. +If there is an HBase gateway on the same host as the Flink gateway where the application is started, this can be obtained by invoking `HBaseConfiguration.create()` as in the examples below. +If that's not the case a configuration should be provided where the proper core-site, hdfs-site, and hbase-site are added as resources. + +## DataStream API + +### Reading data from HBase + +To receive data from HBase, the connector makes use of the internal replication mechanism of HBase. +The connector registers at the HBase cluster as a *Replication Peer* and will receive all change events from HBase. + +For the replication to work, the HBase config needs to have replication enabled in the `hbase-site.xml` file. +This needs be done only once per cluster: +```xml +<configuration> + <property> + <name>hbase.replication</name> + <value>true</value> + </property> + ... +</configuration> +``` +All incoming events to Flink will be processed as an `HBaseSourceEvent`. +You will need to specify a Deserializer which will transform each event from an `HBaseSourceEvent` to the desired DataStream type. + +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +Configuration hbaseConfig = HBaseConfiguration.create(); +String tableName = "TestTable"; + +HBaseSource<String> hbaseSource = + HBaseSource.<String>builder() + .setTableName(tableName) + .setSourceDeserializer(new HBaseStringDeserializer()) + .setHBaseConfiguration(hbaseConfig) + .build(); + +DataStream<String> stream = env.fromSource( + hbaseSource, + WatermarkStrategy.noWatermarks(), + "HBaseSource"); +// ... +``` + +The Deserializer is created as follows: + +```java +static class HBaseStringDeserializer implements HBaseSourceDeserializer<String> { + @Override + public String deserialize(HBaseSourceEvent event) { + return new String(event.getPayload()); Review comment: Please add Charset as second parameter (best practice). ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/writer/HBaseWriter.java ########## @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink.writer; + +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.hbase.HBaseEvent; +import org.apache.flink.connector.hbase.sink.HBaseSinkCommittable; +import org.apache.flink.connector.hbase.sink.HBaseSinkOptions; +import org.apache.flink.connector.hbase.sink.HBaseSinkSerializer; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.flink.shaded.curator4.org.apache.curator.shaded.com.google.common.io.Closer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ArrayBlockingQueue; + +/** HBaseWriter. */ Review comment: Please expand a bit in particular on the flushing side. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/writer/HBaseWriter.java ########## @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink.writer; + +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.hbase.HBaseEvent; +import org.apache.flink.connector.hbase.sink.HBaseSinkCommittable; +import org.apache.flink.connector.hbase.sink.HBaseSinkOptions; +import org.apache.flink.connector.hbase.sink.HBaseSinkSerializer; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.flink.shaded.curator4.org.apache.curator.shaded.com.google.common.io.Closer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ArrayBlockingQueue; + +/** HBaseWriter. */ +public class HBaseWriter<IN> implements SinkWriter<IN, HBaseSinkCommittable, Mutation> { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseWriter.class); + + private final int queueLimit; + private final int maxLatencyMs; + private final HBaseSinkSerializer<IN> sinkSerializer; + private final ArrayBlockingQueue<Mutation> pendingMutations; + private final Connection connection; + private final Table table; + private long lastFlushTimeStamp = 0; + private TimerTask batchSendTimer; + + public HBaseWriter( + Sink.InitContext context, + List<Mutation> states, + HBaseSinkSerializer<IN> sinkSerializer, + byte[] serializedConfig, + Properties properties) { + this.sinkSerializer = sinkSerializer; + this.queueLimit = HBaseSinkOptions.getQueueLimit(properties); + this.maxLatencyMs = HBaseSinkOptions.getMaxLatency(properties); + String tableName = HBaseSinkOptions.getTableName(properties); + + // Queue limit is multiplied by 2, to reduce the probability of blocking while committing + this.pendingMutations = new ArrayBlockingQueue<>(2 * queueLimit); + pendingMutations.addAll(states); + + Configuration hbaseConfiguration = + HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null); + try { + connection = ConnectionFactory.createConnection(hbaseConfiguration); + table = connection.getTable(TableName.valueOf(tableName)); + } catch (IOException e) { + throw new RuntimeException("Connection to HBase couldn't be established", e); + } + + startBatchSendTimer(); + LOG.debug("started sink writer"); + } + + private void startBatchSendTimer() { + batchSendTimer = + new TimerTask() { + @Override + public void run() { + long diff = System.currentTimeMillis() - lastFlushTimeStamp; + if (diff > maxLatencyMs) { + LOG.debug("Time based flushing of mutations"); + flushBuffer(); + } + } + }; + new Timer().scheduleAtFixedRate(batchSendTimer, 0, maxLatencyMs / 2); + } + + private void flushBuffer() { + lastFlushTimeStamp = System.currentTimeMillis(); Review comment: Move this line into `run` to make visibility clearer. Or if it's intended that any batch flush resets the timer, make it `volatile`. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSinkBuilder.java ########## @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink; + +import org.apache.hadoop.conf.Configuration; + +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The builder class to create an {@link HBaseSink}. The following example shows how to create an + * HBaseSink that writes Long values to HBase. + * + * <pre>{@code + * HBaseSink<Long> hbaseSink = + * HBaseSink.<Long>builder() + * .setTableName(tableName) + * .setSinkSerializer(new HBaseLongSerializer()) + * .setHBaseConfiguration(hbaseConfig) + * .build(); + * }</pre> + * + * <p>Here is an example for the Serializer: + * + * <pre>{@code + * static class HBaseLongSerializer implements HBaseSinkSerializer<Long> { + * @Override + * public HBaseEvent serialize(Long event) { + * return HBaseEvent.putWith( // or deleteWith() + * event.toString(), // rowId + * "exampleColumnFamily", // column family + * "exampleQualifier", // qualifier + * Bytes.toBytes(event.toString())); // payload + * } + * } + * }</pre> + * + * <p>A SerializationSchema is always required, as well as a table name to write to and an + * HBaseConfiguration. + * + * <p>By default each HBaseWriter has a queue limit of 1000 entries used for batching. This can be + * changed with {@link #setQueueLimit(int)}. The maximum allowed latency of an event can be set by + * {@link #setMaxLatencyMs(int)}. After the specified elements will be sent to HBase no matter how + * many elements are currently in the batching queue. + */ +public class HBaseSinkBuilder<IN> { + + private static final String[] REQUIRED_CONFIGS = {HBaseSinkOptions.TABLE_NAME.key()}; + private final Properties properties; Review comment: Please consider switching to Flink's `Configuration`. That makes typed access to properties much easier. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/writer/HBaseWriter.java ########## @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink.writer; + +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.hbase.HBaseEvent; +import org.apache.flink.connector.hbase.sink.HBaseSinkCommittable; +import org.apache.flink.connector.hbase.sink.HBaseSinkOptions; +import org.apache.flink.connector.hbase.sink.HBaseSinkSerializer; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.flink.shaded.curator4.org.apache.curator.shaded.com.google.common.io.Closer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ArrayBlockingQueue; + +/** HBaseWriter. */ +public class HBaseWriter<IN> implements SinkWriter<IN, HBaseSinkCommittable, Mutation> { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseWriter.class); + + private final int queueLimit; + private final int maxLatencyMs; + private final HBaseSinkSerializer<IN> sinkSerializer; + private final ArrayBlockingQueue<Mutation> pendingMutations; + private final Connection connection; + private final Table table; + private long lastFlushTimeStamp = 0; + private TimerTask batchSendTimer; + + public HBaseWriter( + Sink.InitContext context, + List<Mutation> states, + HBaseSinkSerializer<IN> sinkSerializer, + byte[] serializedConfig, + Properties properties) { + this.sinkSerializer = sinkSerializer; + this.queueLimit = HBaseSinkOptions.getQueueLimit(properties); + this.maxLatencyMs = HBaseSinkOptions.getMaxLatency(properties); + String tableName = HBaseSinkOptions.getTableName(properties); + + // Queue limit is multiplied by 2, to reduce the probability of blocking while committing + this.pendingMutations = new ArrayBlockingQueue<>(2 * queueLimit); + pendingMutations.addAll(states); + + Configuration hbaseConfiguration = + HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null); + try { + connection = ConnectionFactory.createConnection(hbaseConfiguration); + table = connection.getTable(TableName.valueOf(tableName)); + } catch (IOException e) { + throw new RuntimeException("Connection to HBase couldn't be established", e); + } + + startBatchSendTimer(); + LOG.debug("started sink writer"); + } + + private void startBatchSendTimer() { + batchSendTimer = + new TimerTask() { + @Override + public void run() { + long diff = System.currentTimeMillis() - lastFlushTimeStamp; + if (diff > maxLatencyMs) { + LOG.debug("Time based flushing of mutations"); + flushBuffer(); + } + } + }; + new Timer().scheduleAtFixedRate(batchSendTimer, 0, maxLatencyMs / 2); + } + + private void flushBuffer() { + lastFlushTimeStamp = System.currentTimeMillis(); + if (pendingMutations.size() == 0) { + return; + } + try { + ArrayList<Mutation> batch = new ArrayList<>(); + pendingMutations.drainTo(batch); + table.batch(batch, null); + pendingMutations.clear(); + } catch (IOException | InterruptedException e) { + throw new RuntimeException("Failed storing batch data in HBase", e); + } + } + + @Override + public void write(IN element, Context context) { + HBaseEvent event = sinkSerializer.serialize(element); + if (event.getType() == Cell.Type.Put) { + Put put = new Put(event.getRowId()); + put.addColumn(event.getCf(), event.getQualifier(), event.getPayload()); + pendingMutations.add(put); + } else if (event.getType() == Cell.Type.Delete) { + Delete delete = new Delete(event.getRowId()); + delete.addColumn(event.getCf(), event.getQualifier()); + pendingMutations.add(delete); + } else { + throw new UnsupportedOperationException("event type not supported"); + } + + if (pendingMutations.size() >= queueLimit) { + LOG.debug("Capacity based flushing of mutations"); + flushBuffer(); + } + } + + @Override + public List<HBaseSinkCommittable> prepareCommit(boolean flush) throws IOException { + return Collections.emptyList(); + } + + @Override + public List<Mutation> snapshotState() throws IOException { + LOG.debug("Snapshotting state"); Review comment: Move down and expand `LOG.debug("Snapshotting state with {} items", snapshot.size());` ########## File path: flink-connectors/flink-connector-hbase/README.md ########## @@ -0,0 +1,118 @@ +# Flink HBase Connector + +This module provides connectors that allow Flink to access [HBase](https://hbase.apache.org/) using [CDC](https://en.wikipedia.org/wiki/Change_data_capture). +It supports the new Source and Sink API specified in [FLIP-27](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface) and [FLIP-143](https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API). + +## Installing HBase + +Follow the instructions from the [HBase Quick Start Guide](http://hbase.apache.org/book.html#quickstart) to install HBase. + +*Version Compatibility*: This module is compatible with Apache HBase *2.3.4*. + +## HBase Configuration + +Connecting to HBase always requires a `Configuration` instance. +If there is an HBase gateway on the same host as the Flink gateway where the application is started, this can be obtained by invoking `HBaseConfiguration.create()` as in the examples below. +If that's not the case a configuration should be provided where the proper core-site, hdfs-site, and hbase-site are added as resources. + +## DataStream API + +### Reading data from HBase + +To receive data from HBase, the connector makes use of the internal replication mechanism of HBase. +The connector registers at the HBase cluster as a *Replication Peer* and will receive all change events from HBase. + +For the replication to work, the HBase config needs to have replication enabled in the `hbase-site.xml` file. +This needs be done only once per cluster: +```xml +<configuration> + <property> + <name>hbase.replication</name> + <value>true</value> + </property> + ... +</configuration> +``` +All incoming events to Flink will be processed as an `HBaseSourceEvent`. +You will need to specify a Deserializer which will transform each event from an `HBaseSourceEvent` to the desired DataStream type. + +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +Configuration hbaseConfig = HBaseConfiguration.create(); +String tableName = "TestTable"; + +HBaseSource<String> hbaseSource = + HBaseSource.<String>builder() + .setTableName(tableName) + .setSourceDeserializer(new HBaseStringDeserializer()) + .setHBaseConfiguration(hbaseConfig) + .build(); + +DataStream<String> stream = env.fromSource( + hbaseSource, + WatermarkStrategy.noWatermarks(), + "HBaseSource"); +// ... +``` + +The Deserializer is created as follows: + +```java +static class HBaseStringDeserializer implements HBaseSourceDeserializer<String> { + @Override + public String deserialize(HBaseSourceEvent event) { + return new String(event.getPayload()); + } +} +``` + +### Writing data to HBase +To write data from Flink into HBase, you can use the `HBaseSink`. +Similar to the `HBaseSource` you need to specify a Serializer which knows how to write your DataStream element into HBase. + +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +Configuration hbaseConfig = HBaseConfiguration.create(); +String tableName = "TestTable"; + +DataStream<Long> longStream = env.fromSequence(0, 100); + +HBaseSink<Long> hbaseSink = + HBaseSink.<Long>builder() + .setTableName(tableName) + .setSinkSerializer(new HBaseLongSerializer()) + .setHBaseConfiguration(hbaseConfig) + .build(); + +longStream.sinkTo(hbaseSink); +// ... +``` +An example Serializer is given below. You need to implement the following five methods, so the connector +knows how to save the data to HBase. +```java +static class HBaseLongSerializer implements HBaseSinkSerializer<Long> { + @Override + public HBaseEvent serialize(Long event) { + return HBaseEvent.putWith( // or deleteWith() Review comment: I like the factory method. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/hbaseendpoint/HBaseEndpoint.java ########## @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.source.hbaseendpoint; + +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; +import org.apache.flink.connector.hbase.source.reader.HBaseSourceEvent; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; + +/** Consumer of HBase WAL edits. */ +public class HBaseEndpoint implements ReplicationTargetInterface { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseEndpoint.class); + private final String clusterKey; + /** The id under which the replication target is made known to the source cluster. */ + private final String replicationPeerId; + + private final Configuration hbaseConf; + private final RecoverableZooKeeper zooKeeper; + private final RpcServer rpcServer; + private final FutureCompletingBlockingQueue<HBaseSourceEvent> walEdits; + private final String hostName; + private final String tableName; + private boolean isRunning = false; + + public HBaseEndpoint(byte[] serializedConfig, Properties properties) + throws IOException, KeeperException, InterruptedException { + this(HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null), properties); + } + + public HBaseEndpoint(Configuration hbaseConf, Properties properties) + throws InterruptedException, KeeperException, IOException { + this(UUID.randomUUID().toString().substring(0, 5), hbaseConf, properties); + } + + public HBaseEndpoint(String peerId, Configuration hbaseConf, Properties properties) + throws IOException, KeeperException, InterruptedException { + this.hbaseConf = hbaseConf; + this.clusterKey = peerId + "_clusterKey"; + this.replicationPeerId = peerId; + this.hostName = HBaseSourceOptions.getHostName(properties); + this.tableName = HBaseSourceOptions.getTableName(properties); + int queueCapacity = HBaseSourceOptions.getEndpointQueueCapacity(properties); + this.walEdits = new FutureCompletingBlockingQueue<>(queueCapacity); + + // Setup + zooKeeper = connectToZooKeeper(); + rpcServer = createServer(); + registerAtZooKeeper(); + } + + private RecoverableZooKeeper connectToZooKeeper() throws IOException { + // Using private ZKUtil and RecoverableZooKeeper here, because the stability is so much + // improved + RecoverableZooKeeper zooKeeper = + ZKUtil.connect(hbaseConf, getZookeeperClientAddress(), null); + LOG.debug("Connected to Zookeeper"); + return zooKeeper; + } + + private RpcServer createServer() throws IOException { + Server server = new EmptyHBaseServer(hbaseConf); + InetSocketAddress initialIsa = new InetSocketAddress(hostName, 0); + String name = "regionserver/" + initialIsa.toString(); + + RpcServer.BlockingServiceAndInterface bsai = + new RpcServer.BlockingServiceAndInterface( + AdminProtos.AdminService.newReflectiveBlockingService(this), + BlockingInterface.class); + RpcServer rpcServer = + RpcServerFactory.createRpcServer( + server, + name, + Arrays.asList(bsai), + initialIsa, + hbaseConf, + new FifoRpcScheduler( + hbaseConf, + hbaseConf.getInt("hbase.regionserver.handler.count", 10))); + + rpcServer.start(); + LOG.debug("Started rpc server at {}", initialIsa); + return rpcServer; + } + + private void registerAtZooKeeper() throws KeeperException, InterruptedException { + createZKPath(getBaseString() + "/" + clusterKey, null, CreateMode.PERSISTENT); + createZKPath(getBaseString() + "/" + clusterKey + "/rs", null, CreateMode.PERSISTENT); + + UUID uuid = UUID.nameUUIDFromBytes(Bytes.toBytes(clusterKey)); + createZKPath( + getBaseString() + "/" + clusterKey + "/hbaseid", + Bytes.toBytes(uuid.toString()), + CreateMode.PERSISTENT); + + ServerName serverName = + ServerName.valueOf( + hostName, + rpcServer.getListenerAddress().getPort(), + System.currentTimeMillis()); + createZKPath( + getBaseString() + "/" + clusterKey + "/rs/" + serverName.getServerName(), + null, + CreateMode.EPHEMERAL); + + LOG.debug("Registered rpc server node at zookeeper"); + } + + /** + * Blocks as long as the queue is empty. If the queue isn't empty it will try and get as many + * elements as possible from the queue. It is not guaranteed that at least one element is in the + * returned list. + * + * @return a list of {@link HBaseSourceEvent}. + */ + public List<HBaseSourceEvent> getAll() { + if (!isRunning) { + // Protects from infinite waiting + throw new RuntimeException("Consumer is not running"); + } + List<HBaseSourceEvent> elements = new ArrayList<>(); + HBaseSourceEvent event; + + try { + walEdits.getAvailabilityFuture().get(); + while ((event = walEdits.poll()) != null) { + elements.add(event); + } + return elements; + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Can't retrieve elements from queue", e); + } + } + + public void wakeup() { + walEdits.notifyAvailable(); + } + + public void close() throws InterruptedException { + // TODO unregister replication peer [and delete zookeeper paths] + isRunning = false; + try { + zooKeeper.close(); + LOG.debug("Closed connection to ZooKeeper"); + } finally { + rpcServer.stop(); + LOG.debug("Closed HBase replication target rpc server"); + } + } + + public void startReplication(List<String> columnFamilies) throws IOException { + if (isRunning) { + throw new RuntimeException("HBase replication endpoint is already running"); + } + try (Connection connection = ConnectionFactory.createConnection(hbaseConf); + Admin admin = connection.getAdmin()) { + + ReplicationPeerConfig peerConfig = createPeerConfig(this.tableName, columnFamilies); + if (admin.listReplicationPeers().stream() + .map(ReplicationPeerDescription::getPeerId) + .anyMatch(replicationPeerId::equals)) { + admin.updateReplicationPeerConfig(replicationPeerId, peerConfig); + } else { + admin.addReplicationPeer(replicationPeerId, peerConfig); + } + isRunning = true; + } + } + + @Override + public AdminProtos.ReplicateWALEntryResponse replicateWALEntry( + RpcController controller, AdminProtos.ReplicateWALEntryRequest request) + throws ServiceException { + List<AdminProtos.WALEntry> entries = request.getEntryList(); + CellScanner cellScanner = ((HBaseRpcController) controller).cellScanner(); + + try { + for (final AdminProtos.WALEntry entry : entries) { + final String table = + TableName.valueOf(entry.getKey().getTableName().toByteArray()).toString(); + final int count = entry.getAssociatedCellCount(); + for (int i = 0; i < count; i++) { + if (!cellScanner.advance()) { + throw new ArrayIndexOutOfBoundsException( + "Expected WAL entry to have " + + count + + "elements, but cell scanner did not have cell for index" + + i); + } + + Cell cell = cellScanner.current(); + HBaseSourceEvent event = HBaseSourceEvent.fromCell(table, cell, i); + walEdits.put(0, event); + } + } + } catch (Exception e) { + throw new ServiceException("Could not replicate WAL entry in HBase endpoint", e); + } + + return AdminProtos.ReplicateWALEntryResponse.newBuilder().build(); Review comment: Is this constant? Could be cached (but probably cheap to create). ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSinkMutationSerializer.java ########## @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** This class serializes {@link Mutation}. */ +public class HBaseSinkMutationSerializer implements SimpleVersionedSerializer<Mutation> { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseSinkMutationSerializer.class); + + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(Mutation mutation) throws IOException { + LOG.debug("serializing mutation"); Review comment: `LOG.debug("serializing mutation {}", mutation);`? ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSinkSerializer.java ########## @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink; + +import org.apache.flink.connector.hbase.HBaseEvent; + +import java.io.Serializable; + +/** TODO docs. */ Review comment: Please add. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/writer/HBaseWriter.java ########## @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink.writer; + +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.hbase.HBaseEvent; +import org.apache.flink.connector.hbase.sink.HBaseSinkCommittable; +import org.apache.flink.connector.hbase.sink.HBaseSinkOptions; +import org.apache.flink.connector.hbase.sink.HBaseSinkSerializer; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.flink.shaded.curator4.org.apache.curator.shaded.com.google.common.io.Closer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ArrayBlockingQueue; + +/** HBaseWriter. */ +public class HBaseWriter<IN> implements SinkWriter<IN, HBaseSinkCommittable, Mutation> { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseWriter.class); + + private final int queueLimit; + private final int maxLatencyMs; + private final HBaseSinkSerializer<IN> sinkSerializer; + private final ArrayBlockingQueue<Mutation> pendingMutations; + private final Connection connection; + private final Table table; + private long lastFlushTimeStamp = 0; + private TimerTask batchSendTimer; + + public HBaseWriter( + Sink.InitContext context, + List<Mutation> states, + HBaseSinkSerializer<IN> sinkSerializer, + byte[] serializedConfig, + Properties properties) { + this.sinkSerializer = sinkSerializer; + this.queueLimit = HBaseSinkOptions.getQueueLimit(properties); + this.maxLatencyMs = HBaseSinkOptions.getMaxLatency(properties); + String tableName = HBaseSinkOptions.getTableName(properties); + + // Queue limit is multiplied by 2, to reduce the probability of blocking while committing + this.pendingMutations = new ArrayBlockingQueue<>(2 * queueLimit); + pendingMutations.addAll(states); + + Configuration hbaseConfiguration = + HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null); + try { + connection = ConnectionFactory.createConnection(hbaseConfiguration); + table = connection.getTable(TableName.valueOf(tableName)); + } catch (IOException e) { + throw new RuntimeException("Connection to HBase couldn't be established", e); + } + + startBatchSendTimer(); + LOG.debug("started sink writer"); + } + + private void startBatchSendTimer() { + batchSendTimer = + new TimerTask() { + @Override + public void run() { + long diff = System.currentTimeMillis() - lastFlushTimeStamp; + if (diff > maxLatencyMs) { + LOG.debug("Time based flushing of mutations"); + flushBuffer(); + } + } + }; + new Timer().scheduleAtFixedRate(batchSendTimer, 0, maxLatencyMs / 2); + } + + private void flushBuffer() { + lastFlushTimeStamp = System.currentTimeMillis(); + if (pendingMutations.size() == 0) { + return; + } + try { + ArrayList<Mutation> batch = new ArrayList<>(); + pendingMutations.drainTo(batch); + table.batch(batch, null); + pendingMutations.clear(); + } catch (IOException | InterruptedException e) { + throw new RuntimeException("Failed storing batch data in HBase", e); + } + } + + @Override + public void write(IN element, Context context) { + HBaseEvent event = sinkSerializer.serialize(element); + if (event.getType() == Cell.Type.Put) { + Put put = new Put(event.getRowId()); + put.addColumn(event.getCf(), event.getQualifier(), event.getPayload()); + pendingMutations.add(put); + } else if (event.getType() == Cell.Type.Delete) { + Delete delete = new Delete(event.getRowId()); + delete.addColumn(event.getCf(), event.getQualifier()); + pendingMutations.add(delete); + } else { + throw new UnsupportedOperationException("event type not supported"); + } + + if (pendingMutations.size() >= queueLimit) { + LOG.debug("Capacity based flushing of mutations"); + flushBuffer(); + } + } + + @Override + public List<HBaseSinkCommittable> prepareCommit(boolean flush) throws IOException { + return Collections.emptyList(); + } + + @Override + public List<Mutation> snapshotState() throws IOException { + LOG.debug("Snapshotting state"); + ArrayList<Mutation> snapshot = new ArrayList<>(); + pendingMutations.drainTo(snapshot); + return snapshot; + } + + @Override + public void close() throws Exception { + Closer closer = Closer.create(); Review comment: ``` try (Closer closer = Closer.create()) { closer.register(connection); closer.register(table); closer.register(this::flushBuffer); closer.register(batchSendTimer::cancel); } ``` You register the closeable in the same order as allocations - they are closed in reverse order. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSinkOptions.java ########## @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.sink; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.util.Properties; +import java.util.function.Function; + +/** Configuration options for HBaseSink. */ +public class HBaseSinkOptions { + + public static final ConfigOption<String> TABLE_NAME = + ConfigOptions.key("table_name") + .stringType() + .noDefaultValue() + .withDescription("The table in which the sink will store data."); + + public static final ConfigOption<Integer> QUEUE_LIMIT = + ConfigOptions.key("queue.limit") + .intType() + .defaultValue(1000) + .withDescription("The maximum buffer size before sending data to HBase"); + + public static final ConfigOption<Integer> MAX_LATENCY = + ConfigOptions.key("buffer.timeout.ms") + .intType() + .defaultValue(1000) + .withDescription( + "The maximum time an element stays in the queue before being flushed."); + + @SuppressWarnings("unchecked") + public static <T> T getOption( Review comment: This wouldn't be necessary with `Configuration` instead of `Properties`. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/reader/HBaseSourceDeserializer.java ########## @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.source.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.TypeExtractor; + +import java.io.IOException; +import java.io.Serializable; + +/** Deserialization Interface. */ Review comment: This is a user-facing interface and deserves more documentation. In particular, it would be good to explain when the user has to overwrite `getProducedType`. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/hbaseendpoint/HBaseEndpoint.java ########## @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.source.hbaseendpoint; + +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; +import org.apache.flink.connector.hbase.source.reader.HBaseSourceEvent; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; + +/** Consumer of HBase WAL edits. */ +public class HBaseEndpoint implements ReplicationTargetInterface { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseEndpoint.class); + private final String clusterKey; + /** The id under which the replication target is made known to the source cluster. */ + private final String replicationPeerId; + + private final Configuration hbaseConf; + private final RecoverableZooKeeper zooKeeper; + private final RpcServer rpcServer; + private final FutureCompletingBlockingQueue<HBaseSourceEvent> walEdits; + private final String hostName; + private final String tableName; + private boolean isRunning = false; + + public HBaseEndpoint(byte[] serializedConfig, Properties properties) + throws IOException, KeeperException, InterruptedException { + this(HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null), properties); + } + + public HBaseEndpoint(Configuration hbaseConf, Properties properties) + throws InterruptedException, KeeperException, IOException { + this(UUID.randomUUID().toString().substring(0, 5), hbaseConf, properties); + } + + public HBaseEndpoint(String peerId, Configuration hbaseConf, Properties properties) + throws IOException, KeeperException, InterruptedException { + this.hbaseConf = hbaseConf; + this.clusterKey = peerId + "_clusterKey"; + this.replicationPeerId = peerId; + this.hostName = HBaseSourceOptions.getHostName(properties); + this.tableName = HBaseSourceOptions.getTableName(properties); + int queueCapacity = HBaseSourceOptions.getEndpointQueueCapacity(properties); + this.walEdits = new FutureCompletingBlockingQueue<>(queueCapacity); + + // Setup + zooKeeper = connectToZooKeeper(); + rpcServer = createServer(); + registerAtZooKeeper(); + } Review comment: Are they for testing only? If so, let's add `@VisibleForTesting`. ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/reader/HBaseSourceDeserializer.java ########## @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.source.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.TypeExtractor; + +import java.io.IOException; +import java.io.Serializable; + +/** Deserialization Interface. */ +@FunctionalInterface +public interface HBaseSourceDeserializer<T> extends Serializable, ResultTypeQueryable<T> { Review comment: It would also be awesome to have some factory method for `HBaseSourceDeserializerWrapper` inside this class (you could make the wrapper then package-private). For example: `HBaseSourceDeserializer.valueOnly` (or payloadOnly if this is more common in hbase speak) ########## File path: flink-connectors/flink-connector-hbase/src/test/java/org/apache/flink/connector/hbase/testutil/HBaseTestCluster.java ########## @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.testutil; + +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hbase.thirdparty.com.google.common.io.Closer; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** Provides static access to a {@link MiniHBaseCluster} for testing. */ +public class HBaseTestCluster extends ExternalResource { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseTestCluster.class); + + public static final String COLUMN_FAMILY_BASE = "info"; + public static final String DEFAULT_COLUMN_FAMILY = COLUMN_FAMILY_BASE + 0; + public static final String QUALIFIER_BASE = "qualifier"; + public static final String DEFAULT_QUALIFIER = QUALIFIER_BASE + 0; + + private MiniHBaseCluster cluster; + private Configuration hbaseConf; + private String testFolder; + + public HBaseTestCluster() {} + + /* + * How to use it + */ + public static void main(String[] args) throws Exception { + HBaseTestCluster hbaseTestCluster = new HBaseTestCluster(); + hbaseTestCluster.startCluster(); + hbaseTestCluster.makeTable("tableName"); + // ... + hbaseTestCluster.shutdownCluster(); + } + + public void startCluster() throws IOException, InterruptedException, ExecutionException { + LOG.info("Starting HBase test cluster ..."); + testFolder = Files.createTempDirectory(null).toString(); + + // Fallback for windows users with space in user name, will not work if path contains space. + if (testFolder.contains(" ")) { + testFolder = "/flink-hbase-test-data/"; + } + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("tempusername")); + + hbaseConf = HBaseConfiguration.create(); + hbaseConf.setInt("replication.stats.thread.period.seconds", 5); + hbaseConf.setLong("replication.sleep.before.failover", 2000); + hbaseConf.setInt("replication.source.maxretriesmultiplier", 10); + hbaseConf.setBoolean("hbase.replication", true); + + System.setProperty(HBaseTestingUtility.BASE_TEST_DIRECTORY_KEY, testFolder); + + HBaseTestingUtility utility = new HBaseTestingUtility(hbaseConf); + LOG.info("Testfolder: {}", utility.getDataTestDir().toString()); + try { + cluster = + utility.startMiniCluster( + StartMiniClusterOption.builder().numRegionServers(3).build()); + int numRegionServers = utility.getHBaseCluster().getRegionServerThreads().size(); + LOG.info("Number of region servers: {}", numRegionServers); + LOG.info( + "ZooKeeper client address: {}:{}", + hbaseConf.get("hbase.zookeeper.quorum"), + hbaseConf.get("hbase.zookeeper.property.clientPort")); + LOG.info( + "Master port={}, web UI at port={}", + hbaseConf.get("hbase.master.port"), + hbaseConf.get("hbase.master.info.port")); + + cluster.waitForActiveAndReadyMaster(30 * 1000); + HBaseAdmin.available(hbaseConf); + LOG.info("HBase test cluster up and running ..."); + + } catch (Exception e) { + throw new RuntimeException("Could not start HBase test mini cluster", e); + } + + assert canConnectToCluster(); + } + + public void shutdownCluster() { + LOG.info("Shutting down HBase test cluster"); + try { + try (Closer closer = Closer.create()) { + closer.register(this::clearTables); + closer.register(this::clearReplicationPeers); + } + try { + // Closer is not able to call this method correctly, instead logs process dump + cluster.shutdown(); + } catch (IOException e) { + LOG.error("Error while shutting down HBase test cluster", e); + } + try (Closer closer = Closer.create()) { + closer.register(this::waitForShutDown); + closer.register(Paths.get(testFolder).toFile()::delete); + } + } catch (IOException e) { + throw new RuntimeException( + "Failed to shut down HBase test cluster. Future program state might be compromised.", + e); + } + LOG.info("HBase test cluster shut down"); + } + + public void waitForShutDown() { + try { + CompletableFuture.runAsync(cluster::waitUntilShutDown).get(240, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error("Interrupted while waiting for HBase test cluster to shut down", e); + } catch (ExecutionException e) { + e.printStackTrace(); + LOG.error("Exception while waiting for HBase test cluster to shut down", e); + } catch (TimeoutException e) { + e.printStackTrace(); + LOG.error("Waiting for HBase test cluster to shut down timed out", e); + } + } + + public boolean canConnectToCluster() throws InterruptedException, ExecutionException { + try { + return CompletableFuture.supplyAsync( + () -> { + try (Connection connection = + ConnectionFactory.createConnection(getConfig())) { + return true; + } catch (IOException e) { + LOG.error("Error trying to connect to cluster", e); + return false; + } + }) + .get(10, TimeUnit.SECONDS); + } catch (TimeoutException e) { + LOG.error("Trying to connect to HBase test cluster timed out", e); + return false; + } + } + + public void clearTables() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + for (TableDescriptor table : admin.listTableDescriptors()) { + admin.disableTable(table.getTableName()); + admin.deleteTable(table.getTableName()); + } + } catch (IOException e) { Review comment: You can let `IOExceptions` just bubble up to the `Closer`. If you want to disable/close as many tables as possible, you should add them individually to closer (or create a new one here). ########## File path: flink-connectors/flink-connector-hbase/src/test/java/org/apache/flink/connector/hbase/testutil/HBaseTestCluster.java ########## @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.testutil; + +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hbase.thirdparty.com.google.common.io.Closer; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** Provides static access to a {@link MiniHBaseCluster} for testing. */ +public class HBaseTestCluster extends ExternalResource { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseTestCluster.class); + + public static final String COLUMN_FAMILY_BASE = "info"; + public static final String DEFAULT_COLUMN_FAMILY = COLUMN_FAMILY_BASE + 0; + public static final String QUALIFIER_BASE = "qualifier"; + public static final String DEFAULT_QUALIFIER = QUALIFIER_BASE + 0; + + private MiniHBaseCluster cluster; + private Configuration hbaseConf; + private String testFolder; + + public HBaseTestCluster() {} + + /* + * How to use it + */ + public static void main(String[] args) throws Exception { + HBaseTestCluster hbaseTestCluster = new HBaseTestCluster(); + hbaseTestCluster.startCluster(); + hbaseTestCluster.makeTable("tableName"); + // ... + hbaseTestCluster.shutdownCluster(); + } + + public void startCluster() throws IOException, InterruptedException, ExecutionException { + LOG.info("Starting HBase test cluster ..."); + testFolder = Files.createTempDirectory(null).toString(); + + // Fallback for windows users with space in user name, will not work if path contains space. + if (testFolder.contains(" ")) { + testFolder = "/flink-hbase-test-data/"; + } + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("tempusername")); + + hbaseConf = HBaseConfiguration.create(); + hbaseConf.setInt("replication.stats.thread.period.seconds", 5); + hbaseConf.setLong("replication.sleep.before.failover", 2000); + hbaseConf.setInt("replication.source.maxretriesmultiplier", 10); + hbaseConf.setBoolean("hbase.replication", true); + + System.setProperty(HBaseTestingUtility.BASE_TEST_DIRECTORY_KEY, testFolder); + + HBaseTestingUtility utility = new HBaseTestingUtility(hbaseConf); + LOG.info("Testfolder: {}", utility.getDataTestDir().toString()); + try { + cluster = + utility.startMiniCluster( + StartMiniClusterOption.builder().numRegionServers(3).build()); + int numRegionServers = utility.getHBaseCluster().getRegionServerThreads().size(); + LOG.info("Number of region servers: {}", numRegionServers); + LOG.info( + "ZooKeeper client address: {}:{}", + hbaseConf.get("hbase.zookeeper.quorum"), + hbaseConf.get("hbase.zookeeper.property.clientPort")); + LOG.info( + "Master port={}, web UI at port={}", + hbaseConf.get("hbase.master.port"), + hbaseConf.get("hbase.master.info.port")); + + cluster.waitForActiveAndReadyMaster(30 * 1000); + HBaseAdmin.available(hbaseConf); + LOG.info("HBase test cluster up and running ..."); + + } catch (Exception e) { + throw new RuntimeException("Could not start HBase test mini cluster", e); + } + + assert canConnectToCluster(); + } + + public void shutdownCluster() { + LOG.info("Shutting down HBase test cluster"); + try { + try (Closer closer = Closer.create()) { + closer.register(this::clearTables); + closer.register(this::clearReplicationPeers); + } + try { + // Closer is not able to call this method correctly, instead logs process dump + cluster.shutdown(); + } catch (IOException e) { + LOG.error("Error while shutting down HBase test cluster", e); + } + try (Closer closer = Closer.create()) { + closer.register(this::waitForShutDown); + closer.register(Paths.get(testFolder).toFile()::delete); + } + } catch (IOException e) { + throw new RuntimeException( + "Failed to shut down HBase test cluster. Future program state might be compromised.", + e); + } + LOG.info("HBase test cluster shut down"); + } + + public void waitForShutDown() { + try { + CompletableFuture.runAsync(cluster::waitUntilShutDown).get(240, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error("Interrupted while waiting for HBase test cluster to shut down", e); + } catch (ExecutionException e) { + e.printStackTrace(); + LOG.error("Exception while waiting for HBase test cluster to shut down", e); + } catch (TimeoutException e) { + e.printStackTrace(); + LOG.error("Waiting for HBase test cluster to shut down timed out", e); + } + } + + public boolean canConnectToCluster() throws InterruptedException, ExecutionException { + try { + return CompletableFuture.supplyAsync( + () -> { + try (Connection connection = + ConnectionFactory.createConnection(getConfig())) { + return true; + } catch (IOException e) { + LOG.error("Error trying to connect to cluster", e); + return false; + } + }) + .get(10, TimeUnit.SECONDS); + } catch (TimeoutException e) { + LOG.error("Trying to connect to HBase test cluster timed out", e); + return false; + } + } + + public void clearTables() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + for (TableDescriptor table : admin.listTableDescriptors()) { + admin.disableTable(table.getTableName()); + admin.deleteTable(table.getTableName()); + } + } catch (IOException e) { + throw new RuntimeException("Could not clear test cluster tables", e); + } + } + + public void clearReplicationPeers() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + StringBuilder logMessage = new StringBuilder("Clearing existing replication peers:"); + for (ReplicationPeerDescription desc : admin.listReplicationPeers()) { + logMessage.append("\n\t").append(desc.getPeerId()).append(" | ").append(desc); + admin.removeReplicationPeer(desc.getPeerId()); + } + LOG.info(logMessage.toString()); + } catch (IOException e) { + throw new RuntimeException("Could not clear test cluster replication peers", e); + } + } + + public List<ReplicationPeerDescription> getReplicationPeers() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + return admin.listReplicationPeers(); + } catch (IOException e) { + LOG.error("Error retrieving replication peers", e); + return null; Review comment: Afaik no code path can handle null. It also sounds as if a test failure would be more appropriate for example by throwing `new AssertionError(..., e)`. ########## File path: flink-connectors/flink-connector-hbase/src/test/java/org/apache/flink/connector/hbase/testutil/HBaseTestCluster.java ########## @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.testutil; + +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hbase.thirdparty.com.google.common.io.Closer; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** Provides static access to a {@link MiniHBaseCluster} for testing. */ +public class HBaseTestCluster extends ExternalResource { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseTestCluster.class); + + public static final String COLUMN_FAMILY_BASE = "info"; + public static final String DEFAULT_COLUMN_FAMILY = COLUMN_FAMILY_BASE + 0; + public static final String QUALIFIER_BASE = "qualifier"; + public static final String DEFAULT_QUALIFIER = QUALIFIER_BASE + 0; + + private MiniHBaseCluster cluster; + private Configuration hbaseConf; + private String testFolder; + + public HBaseTestCluster() {} + + /* + * How to use it + */ + public static void main(String[] args) throws Exception { + HBaseTestCluster hbaseTestCluster = new HBaseTestCluster(); + hbaseTestCluster.startCluster(); + hbaseTestCluster.makeTable("tableName"); + // ... + hbaseTestCluster.shutdownCluster(); + } + + public void startCluster() throws IOException, InterruptedException, ExecutionException { + LOG.info("Starting HBase test cluster ..."); + testFolder = Files.createTempDirectory(null).toString(); + + // Fallback for windows users with space in user name, will not work if path contains space. + if (testFolder.contains(" ")) { + testFolder = "/flink-hbase-test-data/"; + } + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("tempusername")); + + hbaseConf = HBaseConfiguration.create(); + hbaseConf.setInt("replication.stats.thread.period.seconds", 5); + hbaseConf.setLong("replication.sleep.before.failover", 2000); + hbaseConf.setInt("replication.source.maxretriesmultiplier", 10); + hbaseConf.setBoolean("hbase.replication", true); + + System.setProperty(HBaseTestingUtility.BASE_TEST_DIRECTORY_KEY, testFolder); + + HBaseTestingUtility utility = new HBaseTestingUtility(hbaseConf); + LOG.info("Testfolder: {}", utility.getDataTestDir().toString()); + try { + cluster = + utility.startMiniCluster( + StartMiniClusterOption.builder().numRegionServers(3).build()); + int numRegionServers = utility.getHBaseCluster().getRegionServerThreads().size(); + LOG.info("Number of region servers: {}", numRegionServers); + LOG.info( + "ZooKeeper client address: {}:{}", + hbaseConf.get("hbase.zookeeper.quorum"), + hbaseConf.get("hbase.zookeeper.property.clientPort")); + LOG.info( + "Master port={}, web UI at port={}", + hbaseConf.get("hbase.master.port"), + hbaseConf.get("hbase.master.info.port")); + + cluster.waitForActiveAndReadyMaster(30 * 1000); + HBaseAdmin.available(hbaseConf); + LOG.info("HBase test cluster up and running ..."); + + } catch (Exception e) { + throw new RuntimeException("Could not start HBase test mini cluster", e); + } + + assert canConnectToCluster(); + } + + public void shutdownCluster() { + LOG.info("Shutting down HBase test cluster"); + try { + try (Closer closer = Closer.create()) { + closer.register(this::clearTables); + closer.register(this::clearReplicationPeers); + } + try { + // Closer is not able to call this method correctly, instead logs process dump + cluster.shutdown(); + } catch (IOException e) { + LOG.error("Error while shutting down HBase test cluster", e); + } + try (Closer closer = Closer.create()) { + closer.register(this::waitForShutDown); + closer.register(Paths.get(testFolder).toFile()::delete); + } Review comment: I guess the close order is not 100% correct yet. The first thing you initialized was the `testFolder`, so that should be added first to `Closer` (and thus will be closed last). Then `waitForShutDown`, `shutdown`, `clear` ... ########## File path: flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/source/hbaseendpoint/HBaseEndpoint.java ########## @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.source.hbaseendpoint; + +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; +import org.apache.flink.connector.hbase.source.reader.HBaseSourceEvent; +import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.ipc.FifoRpcScheduler; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.ipc.RpcServerFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; + +/** Consumer of HBase WAL edits. */ +public class HBaseEndpoint implements ReplicationTargetInterface { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseEndpoint.class); + private final String clusterKey; + /** The id under which the replication target is made known to the source cluster. */ + private final String replicationPeerId; + + private final Configuration hbaseConf; + private final RecoverableZooKeeper zooKeeper; + private final RpcServer rpcServer; + private final FutureCompletingBlockingQueue<HBaseSourceEvent> walEdits; + private final String hostName; + private final String tableName; + private boolean isRunning = false; + + public HBaseEndpoint(byte[] serializedConfig, Properties properties) + throws IOException, KeeperException, InterruptedException { + this(HBaseConfigurationUtil.deserializeConfiguration(serializedConfig, null), properties); + } + + public HBaseEndpoint(Configuration hbaseConf, Properties properties) + throws InterruptedException, KeeperException, IOException { + this(UUID.randomUUID().toString().substring(0, 5), hbaseConf, properties); + } + + public HBaseEndpoint(String peerId, Configuration hbaseConf, Properties properties) + throws IOException, KeeperException, InterruptedException { + this.hbaseConf = hbaseConf; + this.clusterKey = peerId + "_clusterKey"; + this.replicationPeerId = peerId; + this.hostName = HBaseSourceOptions.getHostName(properties); + this.tableName = HBaseSourceOptions.getTableName(properties); + int queueCapacity = HBaseSourceOptions.getEndpointQueueCapacity(properties); + this.walEdits = new FutureCompletingBlockingQueue<>(queueCapacity); + + // Setup + zooKeeper = connectToZooKeeper(); + rpcServer = createServer(); + registerAtZooKeeper(); + } + + private RecoverableZooKeeper connectToZooKeeper() throws IOException { + // Using private ZKUtil and RecoverableZooKeeper here, because the stability is so much + // improved + RecoverableZooKeeper zooKeeper = + ZKUtil.connect(hbaseConf, getZookeeperClientAddress(), null); + LOG.debug("Connected to Zookeeper"); + return zooKeeper; + } + + private RpcServer createServer() throws IOException { + Server server = new EmptyHBaseServer(hbaseConf); + InetSocketAddress initialIsa = new InetSocketAddress(hostName, 0); + String name = "regionserver/" + initialIsa.toString(); + + RpcServer.BlockingServiceAndInterface bsai = + new RpcServer.BlockingServiceAndInterface( + AdminProtos.AdminService.newReflectiveBlockingService(this), + BlockingInterface.class); + RpcServer rpcServer = + RpcServerFactory.createRpcServer( + server, + name, + Arrays.asList(bsai), + initialIsa, + hbaseConf, + new FifoRpcScheduler( + hbaseConf, + hbaseConf.getInt("hbase.regionserver.handler.count", 10))); + + rpcServer.start(); + LOG.debug("Started rpc server at {}", initialIsa); + return rpcServer; + } + + private void registerAtZooKeeper() throws KeeperException, InterruptedException { + createZKPath(getBaseString() + "/" + clusterKey, null, CreateMode.PERSISTENT); + createZKPath(getBaseString() + "/" + clusterKey + "/rs", null, CreateMode.PERSISTENT); + + UUID uuid = UUID.nameUUIDFromBytes(Bytes.toBytes(clusterKey)); + createZKPath( + getBaseString() + "/" + clusterKey + "/hbaseid", + Bytes.toBytes(uuid.toString()), + CreateMode.PERSISTENT); + + ServerName serverName = + ServerName.valueOf( + hostName, + rpcServer.getListenerAddress().getPort(), + System.currentTimeMillis()); + createZKPath( + getBaseString() + "/" + clusterKey + "/rs/" + serverName.getServerName(), + null, + CreateMode.EPHEMERAL); + + LOG.debug("Registered rpc server node at zookeeper"); + } + + /** + * Blocks as long as the queue is empty. If the queue isn't empty it will try and get as many + * elements as possible from the queue. It is not guaranteed that at least one element is in the + * returned list. + * + * @return a list of {@link HBaseSourceEvent}. + */ + public List<HBaseSourceEvent> getAll() { + if (!isRunning) { + // Protects from infinite waiting + throw new RuntimeException("Consumer is not running"); + } + List<HBaseSourceEvent> elements = new ArrayList<>(); + HBaseSourceEvent event; + + try { + walEdits.getAvailabilityFuture().get(); + while ((event = walEdits.poll()) != null) { + elements.add(event); + } + return elements; + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Can't retrieve elements from queue", e); + } + } + + public void wakeup() { + walEdits.notifyAvailable(); + } + + public void close() throws InterruptedException { + // TODO unregister replication peer [and delete zookeeper paths] + isRunning = false; + try { + zooKeeper.close(); + LOG.debug("Closed connection to ZooKeeper"); + } finally { + rpcServer.stop(); + LOG.debug("Closed HBase replication target rpc server"); + } + } + + public void startReplication(List<String> columnFamilies) throws IOException { + if (isRunning) { + throw new RuntimeException("HBase replication endpoint is already running"); + } + try (Connection connection = ConnectionFactory.createConnection(hbaseConf); + Admin admin = connection.getAdmin()) { + + ReplicationPeerConfig peerConfig = createPeerConfig(this.tableName, columnFamilies); + if (admin.listReplicationPeers().stream() + .map(ReplicationPeerDescription::getPeerId) + .anyMatch(replicationPeerId::equals)) { + admin.updateReplicationPeerConfig(replicationPeerId, peerConfig); + } else { + admin.addReplicationPeer(replicationPeerId, peerConfig); + } + isRunning = true; + } + } + + @Override + public AdminProtos.ReplicateWALEntryResponse replicateWALEntry( + RpcController controller, AdminProtos.ReplicateWALEntryRequest request) + throws ServiceException { + List<AdminProtos.WALEntry> entries = request.getEntryList(); + CellScanner cellScanner = ((HBaseRpcController) controller).cellScanner(); + + try { + for (final AdminProtos.WALEntry entry : entries) { + final String table = + TableName.valueOf(entry.getKey().getTableName().toByteArray()).toString(); + final int count = entry.getAssociatedCellCount(); + for (int i = 0; i < count; i++) { + if (!cellScanner.advance()) { + throw new ArrayIndexOutOfBoundsException( + "Expected WAL entry to have " + + count + + "elements, but cell scanner did not have cell for index" + + i); + } + + Cell cell = cellScanner.current(); + HBaseSourceEvent event = HBaseSourceEvent.fromCell(table, cell, i); + walEdits.put(0, event); + } + } + } catch (Exception e) { + throw new ServiceException("Could not replicate WAL entry in HBase endpoint", e); + } + + return AdminProtos.ReplicateWALEntryResponse.newBuilder().build(); + } + + private ReplicationPeerConfig createPeerConfig(String table, List<String> columnFamilies) { + Map<TableName, List<String>> tableCFsMap = new HashMap<>(); + tableCFsMap.put(TableName.valueOf(table), columnFamilies); + return ReplicationPeerConfig.newBuilder() + .setClusterKey( + getZookeeperClientAddress() + ":" + getBaseString() + "/" + clusterKey) + .setReplicateAllUserTables(false) + .setTableCFsMap(tableCFsMap) + .build(); + } + + private String getZookeeperClientAddress() { + return hbaseConf.get("hbase.zookeeper.quorum") + + ":" + + hbaseConf.get("hbase.zookeeper.property.clientPort"); + } + + private String getBaseString() { + // TODO hbasesep.* will never be resolved Review comment: I don't understand this comment. Let's resolve this before merging. ########## File path: flink-connectors/flink-connector-hbase/src/test/java/org/apache/flink/connector/hbase/testutil/HBaseTestCluster.java ########## @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.hbase.testutil; + +import org.apache.flink.connector.hbase.source.HBaseSourceOptions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hbase.thirdparty.com.google.common.io.Closer; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** Provides static access to a {@link MiniHBaseCluster} for testing. */ +public class HBaseTestCluster extends ExternalResource { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseTestCluster.class); + + public static final String COLUMN_FAMILY_BASE = "info"; + public static final String DEFAULT_COLUMN_FAMILY = COLUMN_FAMILY_BASE + 0; + public static final String QUALIFIER_BASE = "qualifier"; + public static final String DEFAULT_QUALIFIER = QUALIFIER_BASE + 0; + + private MiniHBaseCluster cluster; + private Configuration hbaseConf; + private String testFolder; + + public HBaseTestCluster() {} + + /* + * How to use it + */ + public static void main(String[] args) throws Exception { + HBaseTestCluster hbaseTestCluster = new HBaseTestCluster(); + hbaseTestCluster.startCluster(); + hbaseTestCluster.makeTable("tableName"); + // ... + hbaseTestCluster.shutdownCluster(); + } + + public void startCluster() throws IOException, InterruptedException, ExecutionException { + LOG.info("Starting HBase test cluster ..."); + testFolder = Files.createTempDirectory(null).toString(); + + // Fallback for windows users with space in user name, will not work if path contains space. + if (testFolder.contains(" ")) { + testFolder = "/flink-hbase-test-data/"; + } + UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("tempusername")); + + hbaseConf = HBaseConfiguration.create(); + hbaseConf.setInt("replication.stats.thread.period.seconds", 5); + hbaseConf.setLong("replication.sleep.before.failover", 2000); + hbaseConf.setInt("replication.source.maxretriesmultiplier", 10); + hbaseConf.setBoolean("hbase.replication", true); + + System.setProperty(HBaseTestingUtility.BASE_TEST_DIRECTORY_KEY, testFolder); + + HBaseTestingUtility utility = new HBaseTestingUtility(hbaseConf); + LOG.info("Testfolder: {}", utility.getDataTestDir().toString()); + try { + cluster = + utility.startMiniCluster( + StartMiniClusterOption.builder().numRegionServers(3).build()); + int numRegionServers = utility.getHBaseCluster().getRegionServerThreads().size(); + LOG.info("Number of region servers: {}", numRegionServers); + LOG.info( + "ZooKeeper client address: {}:{}", + hbaseConf.get("hbase.zookeeper.quorum"), + hbaseConf.get("hbase.zookeeper.property.clientPort")); + LOG.info( + "Master port={}, web UI at port={}", + hbaseConf.get("hbase.master.port"), + hbaseConf.get("hbase.master.info.port")); + + cluster.waitForActiveAndReadyMaster(30 * 1000); + HBaseAdmin.available(hbaseConf); + LOG.info("HBase test cluster up and running ..."); + + } catch (Exception e) { + throw new RuntimeException("Could not start HBase test mini cluster", e); + } + + assert canConnectToCluster(); + } + + public void shutdownCluster() { + LOG.info("Shutting down HBase test cluster"); + try { + try (Closer closer = Closer.create()) { + closer.register(this::clearTables); + closer.register(this::clearReplicationPeers); + } + try { + // Closer is not able to call this method correctly, instead logs process dump + cluster.shutdown(); + } catch (IOException e) { + LOG.error("Error while shutting down HBase test cluster", e); + } + try (Closer closer = Closer.create()) { + closer.register(this::waitForShutDown); + closer.register(Paths.get(testFolder).toFile()::delete); + } + } catch (IOException e) { + throw new RuntimeException( + "Failed to shut down HBase test cluster. Future program state might be compromised.", + e); + } + LOG.info("HBase test cluster shut down"); + } + + public void waitForShutDown() { + try { + CompletableFuture.runAsync(cluster::waitUntilShutDown).get(240, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error("Interrupted while waiting for HBase test cluster to shut down", e); + } catch (ExecutionException e) { + e.printStackTrace(); + LOG.error("Exception while waiting for HBase test cluster to shut down", e); + } catch (TimeoutException e) { + e.printStackTrace(); + LOG.error("Waiting for HBase test cluster to shut down timed out", e); + } + } + + public boolean canConnectToCluster() throws InterruptedException, ExecutionException { + try { + return CompletableFuture.supplyAsync( + () -> { + try (Connection connection = + ConnectionFactory.createConnection(getConfig())) { + return true; + } catch (IOException e) { + LOG.error("Error trying to connect to cluster", e); + return false; + } + }) + .get(10, TimeUnit.SECONDS); + } catch (TimeoutException e) { + LOG.error("Trying to connect to HBase test cluster timed out", e); + return false; + } + } + + public void clearTables() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + for (TableDescriptor table : admin.listTableDescriptors()) { + admin.disableTable(table.getTableName()); + admin.deleteTable(table.getTableName()); + } + } catch (IOException e) { + throw new RuntimeException("Could not clear test cluster tables", e); + } + } + + public void clearReplicationPeers() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + StringBuilder logMessage = new StringBuilder("Clearing existing replication peers:"); + for (ReplicationPeerDescription desc : admin.listReplicationPeers()) { + logMessage.append("\n\t").append(desc.getPeerId()).append(" | ").append(desc); + admin.removeReplicationPeer(desc.getPeerId()); + } + LOG.info(logMessage.toString()); + } catch (IOException e) { + throw new RuntimeException("Could not clear test cluster replication peers", e); + } + } + + public List<ReplicationPeerDescription> getReplicationPeers() { + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + return admin.listReplicationPeers(); + } catch (IOException e) { + LOG.error("Error retrieving replication peers", e); + return null; + } + } + + public void makeTable(String tableName) { + makeTable(tableName, 1); + } + + /** + * Creates a table for given name with given number of column families. Column family names + * start with {@link HBaseTestCluster#COLUMN_FAMILY_BASE} and are indexed, if more than one is + * requested + */ + public void makeTable(String tableName, int numColumnFamilies) { + assert numColumnFamilies >= 1; + try (Admin admin = ConnectionFactory.createConnection(getConfig()).getAdmin()) { + TableName tableNameObj = TableName.valueOf(tableName); + if (!admin.tableExists(tableNameObj)) { + TableDescriptorBuilder tableBuilder = + TableDescriptorBuilder.newBuilder(tableNameObj); + for (int i = 0; i < numColumnFamilies; i++) { + ColumnFamilyDescriptorBuilder cfBuilder = + ColumnFamilyDescriptorBuilder.newBuilder( + Bytes.toBytes(COLUMN_FAMILY_BASE + i)); + cfBuilder.setScope(1); + tableBuilder.setColumnFamily(cfBuilder.build()); + } + admin.createTable(tableBuilder.build()); + } + } catch (IOException e) { + throw new RuntimeException("Could not create test cluster table", e); Review comment: Maybe it would be easier to just throw the IOException without handling? Alternatively failing the test with an AssertionError. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org