From 6e01c874d186b353d8aae45e5499b842ca51f706 Mon Sep 17 00:00:00 2001 From: Ankit Singhal Date: Mon, 18 Dec 2017 17:55:08 +0530 Subject: [PATCH] TEPHRA-272 Add HBase 2.0 compatibility module --- .travis.yml | 17 +- pom.xml | 2 + .../tephra/util/ConfigurationFactory.java | 7 +- .../org/apache/tephra/util/HBaseVersion.java | 4 + .../util/HBaseVersionSpecificFactory.java | 4 + tephra-examples/hbase-1.4/pom.xml | 2 +- tephra-examples/hbase-2.0/pom.xml | 170 ++ .../apache/tephra/examples/BalanceBooks.java | 341 +++ .../apache/tephra/examples/package-info.java | 40 + .../tephra/examples/BalanceBooksTest.java | 137 ++ tephra-examples/pom.xml | 1 + tephra-hbase-compat-1.4/pom.xml | 24 + tephra-hbase-compat-2.0/pom.xml | 175 ++ .../hbase/HBase20ConfigurationProvider.java | 38 + .../tephra/hbase/SecondaryIndexTable.java | 185 ++ .../tephra/hbase/TransactionAwareHTable.java | 721 +++++++ .../hbase/coprocessor/CellSkipFilter.java | 129 ++ .../coprocessor/FilteredInternalScanner.java | 82 + .../hbase/coprocessor/TransactionFilters.java | 62 + .../coprocessor/TransactionProcessor.java | 574 +++++ .../TransactionVisibilityFilter.java | 305 +++ .../tephra/hbase/txprune/CompactionState.java | 112 + .../hbase/txprune/DataJanitorState.java | 536 +++++ .../HBaseTransactionPruningPlugin.java | 408 ++++ .../txprune/InvalidListPruningDebugTool.java | 429 ++++ .../hbase/txprune/PruneUpperBoundWriter.java | 164 ++ .../PruneUpperBoundWriterSupplier.java | 55 + .../tephra/hbase/txprune/TimeRegions.java | 85 + .../tephra/hbase/AbstractHBaseTableTest.java | 105 + .../HBase20ConfigurationProviderTest.java | 35 + .../hbase/TransactionAwareHTableTest.java | 1861 +++++++++++++++++ .../hbase/coprocessor/CellSkipFilterTest.java | 123 ++ .../coprocessor/TransactionProcessorTest.java | 676 ++++++ .../TransactionVisibilityFilterTest.java | 373 ++++ .../hbase/txprune/DataJanitorStateTest.java | 285 +++ .../hbase/txprune/InvalidListPruneTest.java | 526 +++++ .../txprune/InvalidListPruningDebugTest.java | 432 ++++ .../PruneUpperBoundWriterSupplierTest.java | 122 ++ .../src/test/resources/logback-test.xml | 39 + 39 files changed, 9380 insertions(+), 6 deletions(-) create mode 100644 tephra-examples/hbase-2.0/pom.xml create mode 100644 tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java create mode 100644 tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java create mode 100644 tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java create mode 100644 tephra-hbase-compat-2.0/pom.xml create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/FilteredInternalScanner.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/HBaseTransactionPruningPlugin.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/InvalidListPruningDebugTool.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriter.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplier.java create mode 100644 tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/TimeRegions.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/AbstractHBaseTableTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/HBase20ConfigurationProviderTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/TransactionAwareHTableTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/coprocessor/CellSkipFilterTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionProcessorTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilterTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/txprune/DataJanitorStateTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruneTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/txprune/InvalidListPruningDebugTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/java/org/apache/tephra/hbase/txprune/PruneUpperBoundWriterSupplierTest.java create mode 100644 tephra-hbase-compat-2.0/src/test/resources/logback-test.xml diff --git a/.travis.yml b/.travis.yml index 59a3f202..f5a06b82 100644 --- a/.travis.yml +++ b/.travis.yml @@ -21,6 +21,8 @@ language: java # Run it on precise, until we figure out how to make jdk7 work (or no longer support jdk7). dist: precise +install: true + jdk: - oraclejdk7 - oraclejdk8 @@ -30,10 +32,17 @@ jdk: # Note: -DskipCoreTests will active a profile in tephra-core that skips tests env: - OPTIONS='-am -pl tephra-core' - - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-0.96,tephra-examples/hbase-0.98' - - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-1.0,tephra-examples/hbase-1.0-cdh' - - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-1.1,tephra-examples/hbase-1.2,tephra-examples/cdh-5.7,tephra-examples/cdh-5.8' - - OPTIONS='-am -DskipCoreTests -pl tephra-examples/hbase-1.3,tephra-examples/hbase-1.4' + - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-0.96,tephra-hbase-compat-0.98,tephra-examples/hbase-0.96,tephra-examples/hbase-0.98' + - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.0,tephra-hbase-compat-1.0-cdh,tephra-examples/hbase-1.0,tephra-examples/hbase-1.0-cdh' + - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.1-base,tephra-examples/hbase-1.1,tephra-examples/hbase-1.2,tephra-examples/cdh-5.7,tephra-examples/cdh-5.8' + - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.3,tephra-examples/hbase-1.3' + - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-1.4,tephra-examples/hbase-1.4' + - OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-2.0,tephra-examples/hbase-2.0' + +matrix: + exclude: + - jdk: oraclejdk7 + env: OPTIONS='-am -DskipCoreTests -pl tephra-hbase-compat-2.0,tephra-examples/hbase-2.0' branches: only: diff --git a/pom.xml b/pom.xml index 072df6bd..0975b92b 100644 --- a/pom.xml +++ b/pom.xml @@ -159,6 +159,7 @@ tephra-hbase-compat-1.1-base tephra-hbase-compat-1.3 tephra-hbase-compat-1.4 + tephra-hbase-compat-2.0 tephra-examples tephra-distribution @@ -217,6 +218,7 @@ 1.2.0 1.3.1 1.4.0 + 2.0.0-beta-1 4.11 1.7.5 0.9.0 diff --git a/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java b/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java index 42b57d36..3b39f760 100644 --- a/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java +++ b/tephra-core/src/main/java/org/apache/tephra/util/ConfigurationFactory.java @@ -51,7 +51,12 @@ protected String getHBase11Classname() { protected String getHBase13Classname() { return "org.apache.tephra.hbase.HBase13ConfigurationProvider"; } - + + @Override + protected String getHBase20Classname() { + return "org.apache.tephra.hbase.HBase20ConfigurationProvider"; + } + @Override protected String getHBase10CDHClassname() { return "org.apache.tephra.hbase.HBase10ConfigurationProvider"; diff --git a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java index 84fee8ec..d9bec3bc 100644 --- a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java +++ b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersion.java @@ -39,6 +39,7 @@ public class HBaseVersion { private static final String HBASE_12_VERSION = "1.2"; private static final String HBASE_13_VERSION = "1.3"; private static final String HBASE_14_VERSION = "1.4"; + private static final String HBASE_20_VERSION = "2.0"; private static final String CDH_CLASSIFIER = "cdh"; private static final Logger LOG = LoggerFactory.getLogger(HBaseVersion.class); @@ -56,6 +57,7 @@ public enum Version { HBASE_12("1.2"), HBASE_13("1.3"), HBASE_14("1.4"), + HBASE_20("2.0"), UNKNOWN("unknown"); final String majorVersion; @@ -97,6 +99,8 @@ public String getMajorVersion() { currentVersion = Version.HBASE_13; } else if (versionString.startsWith(HBASE_14_VERSION)) { currentVersion = Version.HBASE_14; + } else if (versionString.startsWith(HBASE_20_VERSION)) { + currentVersion = Version.HBASE_20; } else { currentVersion = Version.UNKNOWN; } diff --git a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java index 685763d7..8beef1f2 100644 --- a/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java +++ b/tephra-core/src/main/java/org/apache/tephra/util/HBaseVersionSpecificFactory.java @@ -58,6 +58,9 @@ public T get() { case HBASE_14: instance = createInstance(getHBase14Classname()); break; + case HBASE_20: + instance = createInstance(getHBase20Classname()); + break; case UNKNOWN: default: throw new ProvisionException("Unknown HBase version: " + HBaseVersion.getVersionString()); @@ -80,4 +83,5 @@ protected T createInstance(String className) throws ClassNotFoundException { protected abstract String getHBase11Classname(); protected abstract String getHBase13Classname(); protected abstract String getHBase14Classname(); + protected abstract String getHBase20Classname(); } diff --git a/tephra-examples/hbase-1.4/pom.xml b/tephra-examples/hbase-1.4/pom.xml index 217631cc..a9fce192 100644 --- a/tephra-examples/hbase-1.4/pom.xml +++ b/tephra-examples/hbase-1.4/pom.xml @@ -41,7 +41,7 @@ org.apache.tephra - tephra-hbase-compat-1.3 + tephra-hbase-compat-1.4 ${project.version} diff --git a/tephra-examples/hbase-2.0/pom.xml b/tephra-examples/hbase-2.0/pom.xml new file mode 100644 index 00000000..e3fc693d --- /dev/null +++ b/tephra-examples/hbase-2.0/pom.xml @@ -0,0 +1,170 @@ + + + + + + org.apache.tephra + tephra-examples + 0.14.0-incubating-SNAPSHOT + + 4.0.0 + + tephra-examples-hbase-2.0 + Apache Tephra Examples For HBase 2.0 + + + 2.7.4 + + + + src/main/java + src/test/java + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + + + + + org.apache.tephra + tephra-hbase-compat-2.0 + ${project.version} + + + org.apache.hbase + hbase-common + ${hbase2.version} + provided + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-client + ${hbase2.version} + provided + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-protocol + ${hbase2.version} + provided + + + org.apache.hbase + hbase-server + ${hbase2.version} + provided + + + org.slf4j + slf4j-log4j12 + + + + + + + org.apache.hbase + hbase-server + ${hbase2.version} + test-jar + test + + + org.slf4j + slf4j-log4j12 + + + + + org.slf4j + slf4j-log4j12 + 1.6.4 + test + + + log4j + log4j + 1.2.17 + test + + + org.slf4j + log4j-over-slf4j + + + org.slf4j + jcl-over-slf4j + + + org.apache.hbase + hbase-testing-util + ${hbase2.version} + test + + + asm + asm + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-testing-util + ${hbase2.version} + test + + + asm + asm + + + + + org.apache.hbase + hbase-http + ${hbase2.version} + test + + + diff --git a/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java new file mode 100644 index 00000000..b9705982 --- /dev/null +++ b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/BalanceBooks.java @@ -0,0 +1,341 @@ +/* + * 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.tephra.examples; + +import com.google.common.io.Closeables; +import com.google.inject.Guice; +import com.google.inject.Injector; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +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.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.tephra.TransactionConflictException; +import org.apache.tephra.TransactionContext; +import org.apache.tephra.TransactionFailureException; +import org.apache.tephra.TransactionSystemClient; +import org.apache.tephra.distributed.TransactionServiceClient; +import org.apache.tephra.hbase.TransactionAwareHTable; +import org.apache.tephra.hbase.coprocessor.TransactionProcessor; +import org.apache.tephra.runtime.ConfigModule; +import org.apache.tephra.runtime.DiscoveryModules; +import org.apache.tephra.runtime.TransactionClientModule; +import org.apache.tephra.runtime.TransactionModules; +import org.apache.tephra.runtime.ZKModule; +import org.apache.tephra.util.ConfigurationFactory; +import org.apache.twill.zookeeper.ZKClientService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +/** + * Simple example application that launches a number of concurrent clients, one per "account". Each client attempts to + * make withdrawals from other clients, and deposit the same amount to its own account in a single transaction. + * Since this means the client will be updating both its own row and the withdrawee's row, this will naturally lead to + * transaction conflicts. All clients will run for a specified number of iterations. When the processing is complete, + * the total sum of all rows should be zero, if transactional integrity was maintained. + * + *

+ * You can run the BalanceBooks application with the following command: + *

+ *     ./bin/tephra run org.apache.tephra.examples.BalanceBooks [num clients] [num iterations]
+ *   
+ * where [num clients] is the number of concurrent client threads to use, and + * [num iterations] is the number of "transfer" operations to perform per client thread. + *

+ */ +public class BalanceBooks implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(BalanceBooks.class); + + private static final int MAX_AMOUNT = 100; + private static final byte[] TABLE = Bytes.toBytes("testbalances"); + private static final byte[] FAMILY = Bytes.toBytes("f"); + private static final byte[] COL = Bytes.toBytes("b"); + + private final int totalClients; + private final int iterations; + + private Configuration conf; + private ZKClientService zkClient; + private TransactionServiceClient txClient; + private Connection conn; + + public BalanceBooks(int totalClients, int iterations) { + this(totalClients, iterations, new ConfigurationFactory().get()); + } + + public BalanceBooks(int totalClients, int iterations, Configuration conf) { + this.totalClients = totalClients; + this.iterations = iterations; + this.conf = conf; + } + + /** + * Sets up common resources required by all clients. + */ + public void init() throws IOException { + Injector injector = Guice.createInjector( + new ConfigModule(conf), + new ZKModule(), + new DiscoveryModules().getDistributedModules(), + new TransactionModules().getDistributedModules(), + new TransactionClientModule() + ); + + zkClient = injector.getInstance(ZKClientService.class); + zkClient.startAndWait(); + txClient = injector.getInstance(TransactionServiceClient.class); + conn = ConnectionFactory.createConnection(conf); + createTableIfNotExists(conf, TABLE, new byte[][]{ FAMILY }); + } + + /** + * Runs all clients and waits for them to complete. + */ + public void run() throws IOException, InterruptedException { + List clients = new ArrayList<>(totalClients); + for (int i = 0; i < totalClients; i++) { + Client c = new Client(i, totalClients, iterations); + c.init(txClient, conn.getTable(TableName.valueOf(TABLE))); + c.start(); + clients.add(c); + } + + for (Client c : clients) { + c.join(); + Closeables.closeQuietly(c); + } + } + + /** + * Validates the current state of the data stored at the end of the test. Each update by a client consists of two + * parts: a withdrawal of a random amount from a randomly select other account, and a corresponding to deposit to + * the client's own account. So, if all the updates were performed consistently (no partial updates or partial + * rollbacks), then the total sum of all balances at the end should be 0. + */ + public boolean verify() { + boolean success = false; + try { + TransactionAwareHTable table = new TransactionAwareHTable(conn.getTable(TableName.valueOf(TABLE))); + TransactionContext context = new TransactionContext(txClient, table); + + LOG.info("VERIFYING BALANCES"); + context.start(); + long totalBalance = 0; + + try (ResultScanner scanner = table.getScanner(new Scan())) { + for (Result r : scanner) { + if (!r.isEmpty()) { + int rowId = Bytes.toInt(r.getRow()); + long balance = Bytes.toLong(r.getValue(FAMILY, COL)); + totalBalance += balance; + LOG.info("Client #{}: balance = ${}", rowId, balance); + } + } + } + if (totalBalance == 0) { + LOG.info("PASSED!"); + success = true; + } else { + LOG.info("FAILED! Total balance should be 0 but was {}", totalBalance); + } + context.finish(); + } catch (Exception e) { + LOG.error("Failed verification check", e); + } + return success; + } + + /** + * Frees up the underlying resources common to all clients. + */ + public void close() { + try { + if (conn != null) { + conn.close(); + } + } catch (IOException ignored) { } + + if (zkClient != null) { + zkClient.stopAndWait(); + } + } + + protected void createTableIfNotExists(Configuration conf, byte[] tableName, byte[][] columnFamilies) + throws IOException { + try (Admin admin = this.conn.getAdmin()) { + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + for (byte[] family : columnFamilies) { + HColumnDescriptor columnDesc = new HColumnDescriptor(family); + columnDesc.setMaxVersions(Integer.MAX_VALUE); + desc.addFamily(columnDesc); + } + desc.addCoprocessor(TransactionProcessor.class.getName()); + admin.createTable(desc); + } + } + + public static void main(String[] args) { + if (args.length != 2) { + System.err.println("Usage: java " + BalanceBooks.class.getName() + " "); + System.err.println("\twhere >= 2"); + System.exit(1); + } + + try (BalanceBooks bb = new BalanceBooks(Integer.parseInt(args[0]), Integer.parseInt(args[1]))) { + bb.init(); + bb.run(); + bb.verify(); + } catch (Exception e) { + LOG.error("Failed during BalanceBooks run", e); + } + } + + /** + * Represents a single client actor in the test. Each client runs as a separate thread. + * + * For the given number of iterations, the client will: + *
    + *
  1. select a random other client from which to withdraw
  2. + *
  3. select a random amount from 0 to MAX_AMOUNT
  4. + *
  5. start a new transaction and: deduct the amount from the other client's acccount, and deposit + * the same amount to its own account.
  6. + *
+ * + * Since multiple clients operate concurrently and contend over a set of constrained resources + * (the client accounts), it is expected that a portion of the attempted transactions will encounter + * conflicts, due to a simultaneous deduction from or deposit to one the same accounts which has successfully + * committed first. In this case, the updates from the transaction encountering the conflict should be completely + * rolled back, leaving the data in a consistent state. + */ + private static class Client extends Thread implements Closeable { + private final int id; + private final int totalClients; + private final int iterations; + + private final Random random = new Random(); + + private TransactionContext txContext; + private TransactionAwareHTable txTable; + + + public Client(int id, int totalClients, int iterations) { + this.id = id; + this.totalClients = totalClients; + this.iterations = iterations; + } + + /** + * Sets up any resources needed by the individual client. + * + * @param txClient the transaction client to use in accessing the transaciton service + * @param table the HBase table instance to use for accessing storage + */ + public void init(TransactionSystemClient txClient, Table table) { + txTable = new TransactionAwareHTable(table); + txContext = new TransactionContext(txClient, txTable); + } + + public void run() { + try { + for (int i = 0; i < iterations; i++) { + runOnce(); + } + } catch (TransactionFailureException e) { + LOG.error("Client #{}: Failed on exception", id, e); + } + } + + /** + * Runs a single iteration of the client logic. + */ + private void runOnce() throws TransactionFailureException { + int withdrawee = getNextWithdrawee(); + int amount = getAmount(); + + try { + txContext.start(); + long withdraweeBalance = getCurrentBalance(withdrawee); + long ownBalance = getCurrentBalance(id); + long withdraweeNew = withdraweeBalance - amount; + long ownNew = ownBalance + amount; + + setBalance(withdrawee, withdraweeNew); + setBalance(id, ownNew); + LOG.debug("Client #{}: Withdrew ${} from #{}; withdrawee old={}, new={}; own old={}, new={}", + id, amount, withdrawee, withdraweeBalance, withdraweeNew, ownBalance, ownNew); + txContext.finish(); + + } catch (IOException ioe) { + LOG.error("Client #{}: Unhandled client failure", id, ioe); + txContext.abort(); + } catch (TransactionConflictException tce) { + LOG.debug("CONFLICT: client #{} attempting to withdraw from #{}", id, withdrawee); + txContext.abort(tce); + } catch (TransactionFailureException tfe) { + LOG.error("Client #{}: Unhandled transaction failure", id, tfe); + txContext.abort(tfe); + } + } + + private long getCurrentBalance(int id) throws IOException { + Result r = txTable.get(new Get(Bytes.toBytes(id))); + byte[] balanceBytes = r.getValue(FAMILY, COL); + if (balanceBytes == null) { + return 0; + } + return Bytes.toLong(balanceBytes); + } + + private void setBalance(int id, long balance) throws IOException { + txTable.put(new Put(Bytes.toBytes(id)).addColumn(FAMILY, COL, Bytes.toBytes(balance))); + } + + private int getNextWithdrawee() { + int next; + do { + next = random.nextInt(totalClients); + } while (next == id); + return next; + } + + private int getAmount() { + return random.nextInt(MAX_AMOUNT); + } + + public void close() throws IOException { + txTable.close(); + } + } +} diff --git a/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java new file mode 100644 index 00000000..a0e67d59 --- /dev/null +++ b/tephra-examples/hbase-2.0/src/main/java/org/apache/tephra/examples/package-info.java @@ -0,0 +1,40 @@ +/* + * 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. + */ + +/** + * This package contains example applications for Tephra designed to illustrate sample Tephra usage + * and provide out-of-the-box sample applications which can be run to test cluster functionality. + * + *

Currently the following applications are provided: + * + *

    + *
  • BalanceBooks - this application runs a specified number of concurrent clients in separate + * threads, which perform transactions to make withdrawals from each other's accounts and deposits to their own + * accounts. At the end of the test, the total value of all account balances is verified to be equal to zero, + * which confirms that transactional integrity was not violated. + *
  • + *
+ *

+ * + *

+ * Note that, for simplicity, the examples package is currently hardcoded to compile against a specific HBase + * version (currently 2.0). In the future, we should provide Maven profiles to allow compiling the examples + * against each of the supported HBase versions. + *

+ */ +package org.apache.tephra.examples; diff --git a/tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java b/tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java new file mode 100644 index 00000000..4dfe1072 --- /dev/null +++ b/tephra-examples/hbase-2.0/src/test/java/org/apache/tephra/examples/BalanceBooksTest.java @@ -0,0 +1,137 @@ +/* + * 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.tephra.examples; + +import com.google.inject.AbstractModule; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Scopes; +import com.google.inject.util.Modules; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.tephra.TransactionSystemClient; +import org.apache.tephra.TxConstants; +import org.apache.tephra.distributed.TransactionService; +import org.apache.tephra.persist.InMemoryTransactionStateStorage; +import org.apache.tephra.persist.TransactionStateStorage; +import org.apache.tephra.runtime.ConfigModule; +import org.apache.tephra.runtime.DiscoveryModules; +import org.apache.tephra.runtime.TransactionClientModule; +import org.apache.tephra.runtime.TransactionModules; +import org.apache.tephra.runtime.ZKModule; +import org.apache.tephra.util.Tests; +import org.apache.twill.zookeeper.ZKClientService; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertTrue; + +/** + * Tests the {@link BalanceBooks} program. + */ +public class BalanceBooksTest { + private static final Logger LOG = LoggerFactory.getLogger(BalanceBooksTest.class); + private static HBaseTestingUtility testUtil; + private static TransactionService txService; + private static ZKClientService zkClientService; + + @ClassRule + public static TemporaryFolder tmpFolder = new TemporaryFolder(); + + @BeforeClass + public static void setup() throws Exception { + testUtil = new HBaseTestingUtility(); + Configuration conf = testUtil.getConfiguration(); + conf.setBoolean(TxConstants.Manager.CFG_DO_PERSIST, false); + conf.set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, tmpFolder.newFolder().getAbsolutePath()); + + // Tune down the connection thread pool size + conf.setInt("hbase.hconnection.threads.core", 5); + conf.setInt("hbase.hconnection.threads.max", 10); + // Tunn down handler threads in regionserver + conf.setInt("hbase.regionserver.handler.count", 10); + + // Set to random port + conf.setInt("hbase.master.port", 0); + conf.setInt("hbase.master.info.port", 0); + conf.setInt("hbase.regionserver.port", 0); + conf.setInt("hbase.regionserver.info.port", 0); + + testUtil.startMiniCluster(); + + String zkClusterKey = testUtil.getClusterKey(); // hostname:clientPort:parentZnode + String zkQuorum = zkClusterKey.substring(0, zkClusterKey.lastIndexOf(':')); + LOG.info("Zookeeper Quorum is running at {}", zkQuorum); + conf.set(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM, zkQuorum); + + Injector injector = Guice.createInjector( + new ConfigModule(conf), + new ZKModule(), + new DiscoveryModules().getDistributedModules(), + Modules.override(new TransactionModules().getDistributedModules()) + .with(new AbstractModule() { + @Override + protected void configure() { + bind(TransactionStateStorage.class).to(InMemoryTransactionStateStorage.class).in(Scopes.SINGLETON); + } + }), + new TransactionClientModule() + ); + + zkClientService = injector.getInstance(ZKClientService.class); + zkClientService.startAndWait(); + + // start a tx server + txService = injector.getInstance(TransactionService.class); + try { + LOG.info("Starting transaction service"); + txService.startAndWait(); + } catch (Exception e) { + LOG.error("Failed to start service: ", e); + throw e; + } + + Tests.waitForTxReady(injector.getInstance(TransactionSystemClient.class)); + } + + @AfterClass + public static void tearDown() throws Exception { + if (txService != null) { + txService.stopAndWait(); + } + if (zkClientService != null) { + zkClientService.stopAndWait(); + } + testUtil.shutdownMiniCluster(); + } + + @Test + public void testBalanceBooks() throws Exception { + try (BalanceBooks bb = new BalanceBooks(5, 100, testUtil.getConfiguration())) { + bb.init(); + bb.run(); + assertTrue(bb.verify()); + } + } +} diff --git a/tephra-examples/pom.xml b/tephra-examples/pom.xml index 6f10d3b9..1f84e1e7 100644 --- a/tephra-examples/pom.xml +++ b/tephra-examples/pom.xml @@ -38,6 +38,7 @@ hbase-1.2 hbase-1.3 hbase-1.4 + hbase-2.0 cdh-5.7 cdh-5.8 diff --git a/tephra-hbase-compat-1.4/pom.xml b/tephra-hbase-compat-1.4/pom.xml index d1ee1088..873585d4 100644 --- a/tephra-hbase-compat-1.4/pom.xml +++ b/tephra-hbase-compat-1.4/pom.xml @@ -125,6 +125,30 @@
+ + org.apache.hbase + hbase-metrics-api + ${hbase14.version} + test + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-metrics + ${hbase14.version} + test + + + org.slf4j + slf4j-log4j12 + + + org.apache.hbase hbase-testing-util diff --git a/tephra-hbase-compat-2.0/pom.xml b/tephra-hbase-compat-2.0/pom.xml new file mode 100644 index 00000000..1d4ffa0d --- /dev/null +++ b/tephra-hbase-compat-2.0/pom.xml @@ -0,0 +1,175 @@ + + + + + + org.apache.tephra + tephra + 0.14.0-incubating-SNAPSHOT + + 4.0.0 + + tephra-hbase-compat-2.0 + Apache Tephra HBase 2.0 Compatibility + + + 2.7.4 + 2.0.0-beta-1 + + + + + org.apache.tephra + tephra-api + ${project.version} + + + org.apache.tephra + tephra-core + ${project.version} + + + org.apache.hbase + hbase + + + + + org.apache.hbase + hbase-common + ${hbase20.version} + provided + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-client + ${hbase20.version} + provided + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-protocol + ${hbase20.version} + provided + + + org.apache.hbase + hbase-server + ${hbase20.version} + provided + + + org.slf4j + slf4j-log4j12 + + + + + + + org.apache.tephra + tephra-core + ${project.version} + test-jar + test + + + junit + junit + + + org.slf4j + log4j-over-slf4j + + + org.slf4j + jcl-over-slf4j + + + org.apache.hbase + hbase-server + ${hbase20.version} + test-jar + test + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-zookeeper + ${hbase20.version} + test-jar + test + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hbase + hbase-testing-util + ${hbase20.version} + test + + + asm + asm + + + + + org.apache.hbase + hbase-http + ${hbase20.version} + test + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.8 + 1.8 + + + + + diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.java new file mode 100644 index 00000000..96e4be33 --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/HBase20ConfigurationProvider.java @@ -0,0 +1,38 @@ +/* + * 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.tephra.hbase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.tephra.util.ConfigurationProvider; + +/** + * HBase 2.0 version of {@link ConfigurationProvider}. + */ +public class HBase20ConfigurationProvider extends ConfigurationProvider { + @Override + public Configuration get() { + return HBaseConfiguration.create(); + } + + @Override + public Configuration get(Configuration baseConf) { + return HBaseConfiguration.create(baseConf); + } +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java new file mode 100644 index 00000000..d5163341 --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/SecondaryIndexTable.java @@ -0,0 +1,185 @@ +/* + * 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.tephra.hbase; + +import com.google.common.base.Throwables; +import com.google.common.io.Closeables; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +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.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.tephra.TransactionContext; +import org.apache.tephra.TransactionFailureException; +import org.apache.tephra.distributed.TransactionServiceClient; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A Transactional SecondaryIndexTable. + */ +public class SecondaryIndexTable implements Closeable { + private byte[] secondaryIndex; + private TransactionAwareHTable transactionAwareHTable; + private TransactionAwareHTable secondaryIndexTable; + private TransactionContext transactionContext; + private final TableName secondaryIndexTableName; + private Connection connection; + private static final byte[] secondaryIndexFamily = Bytes.toBytes("secondaryIndexFamily"); + private static final byte[] secondaryIndexQualifier = Bytes.toBytes('r'); + private static final byte[] DELIMITER = new byte[] {0}; + + public SecondaryIndexTable(TransactionServiceClient transactionServiceClient, Table table, + byte[] secondaryIndex) throws IOException { + secondaryIndexTableName = TableName.valueOf(table.getName().getNameAsString() + ".idx"); + this.connection = ConnectionFactory.createConnection(table.getConfiguration()); + Table secondaryIndexHTable = null; + try (Admin hBaseAdmin = this.connection.getAdmin()) { + if (!hBaseAdmin.tableExists(secondaryIndexTableName)) { + hBaseAdmin.createTable(TableDescriptorBuilder.newBuilder(secondaryIndexTableName).build()); + } + secondaryIndexHTable = this.connection.getTable(secondaryIndexTableName); + } catch (Exception e) { + Closeables.closeQuietly(connection); + Throwables.propagate(e); + } + + this.secondaryIndex = secondaryIndex; + this.transactionAwareHTable = new TransactionAwareHTable(table); + this.secondaryIndexTable = new TransactionAwareHTable(secondaryIndexHTable); + this.transactionContext = new TransactionContext(transactionServiceClient, transactionAwareHTable, + secondaryIndexTable); + } + + public Result get(Get get) throws IOException { + return get(Collections.singletonList(get))[0]; + } + + public Result[] get(List gets) throws IOException { + try { + transactionContext.start(); + Result[] result = transactionAwareHTable.get(gets); + transactionContext.finish(); + return result; + } catch (Exception e) { + try { + transactionContext.abort(); + } catch (TransactionFailureException e1) { + throw new IOException("Could not rollback transaction", e1); + } + } + return null; + } + + public Result[] getByIndex(byte[] value) throws IOException { + try { + transactionContext.start(); + Scan scan = new Scan(value, Bytes.add(value, new byte[0])); + scan.addColumn(secondaryIndexFamily, secondaryIndexQualifier); + ResultScanner indexScanner = secondaryIndexTable.getScanner(scan); + + ArrayList gets = new ArrayList<>(); + for (Result result : indexScanner) { + for (Cell cell : result.listCells()) { + gets.add(new Get(CellUtil.cloneValue(cell))); + } + } + Result[] results = transactionAwareHTable.get(gets); + transactionContext.finish(); + return results; + } catch (Exception e) { + try { + transactionContext.abort(); + } catch (TransactionFailureException e1) { + throw new IOException("Could not rollback transaction", e1); + } + } + return null; + } + + public void put(Put put) throws IOException { + put(Collections.singletonList(put)); + } + + + public void put(List puts) throws IOException { + try { + transactionContext.start(); + ArrayList secondaryIndexPuts = new ArrayList<>(); + for (Put put : puts) { + List indexPuts = new ArrayList<>(); + Set>> familyMap = put.getFamilyCellMap().entrySet(); + for (Map.Entry> family : familyMap) { + for (Cell value : family.getValue()) { + if (Bytes.equals(value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(), + secondaryIndex, 0, secondaryIndex.length)) { + byte[] secondaryRow = Bytes.add(CellUtil.cloneQualifier(value), DELIMITER, + Bytes.add(CellUtil.cloneValue(value), DELIMITER, + CellUtil.cloneRow(value))); + Put indexPut = new Put(secondaryRow); + indexPut.addColumn(secondaryIndexFamily, secondaryIndexQualifier, put.getRow()); + indexPuts.add(indexPut); + } + } + } + secondaryIndexPuts.addAll(indexPuts); + } + transactionAwareHTable.put(puts); + secondaryIndexTable.put(secondaryIndexPuts); + transactionContext.finish(); + } catch (Exception e) { + try { + transactionContext.abort(); + } catch (TransactionFailureException e1) { + throw new IOException("Could not rollback transaction", e1); + } + } + } + + @Override + public void close() throws IOException { + Closeables.closeQuietly(connection); + try { + transactionAwareHTable.close(); + } catch (IOException e) { + try { + secondaryIndexTable.close(); + } catch (IOException ex) { + e.addSuppressed(ex); + } + throw e; + } + secondaryIndexTable.close(); + } +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java new file mode 100644 index 00000000..1f59ea0b --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/TransactionAwareHTable.java @@ -0,0 +1,721 @@ +/* + * 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.tephra.hbase; + +import com.google.protobuf.Descriptors.MethodDescriptor; +import com.google.protobuf.Message; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.OperationWithAttributes; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Row; +import org.apache.hadoop.hbase.client.RowMutations; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; +import org.apache.hadoop.hbase.filter.CompareFilter; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; +import org.apache.tephra.AbstractTransactionAwareTable; +import org.apache.tephra.Transaction; +import org.apache.tephra.TransactionAware; +import org.apache.tephra.TxConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * A Transaction Aware HTable implementation for HBase 2.0. Operations are committed as usual, but + * upon a failed or aborted transaction, they are rolled back to the state before the transaction + * was started. + */ +public class TransactionAwareHTable extends AbstractTransactionAwareTable + implements Table, TransactionAware { + + private static final Logger LOG = LoggerFactory.getLogger(TransactionAwareHTable.class); + private final Table hTable; + + /** + * Create a transactional aware instance of the passed HTable + * @param hTable underlying HBase table to use + */ + public TransactionAwareHTable(Table hTable) { + this(hTable, false); + } + + /** + * Create a transactional aware instance of the passed HTable + * @param hTable underlying HBase table to use + * @param conflictLevel level of conflict detection to perform (defaults to {@code COLUMN}) + */ + public TransactionAwareHTable(Table hTable, TxConstants.ConflictDetection conflictLevel) { + this(hTable, conflictLevel, false); + } + + /** + * Create a transactional aware instance of the passed HTable, with the option of allowing + * non-transactional operations. + * @param hTable underlying HBase table to use + * @param allowNonTransactional if true, additional operations (checkAndPut, increment, + * checkAndDelete) will be available, though non-transactional + */ + public TransactionAwareHTable(Table hTable, boolean allowNonTransactional) { + this(hTable, TxConstants.ConflictDetection.COLUMN, allowNonTransactional); + } + + /** + * Create a transactional aware instance of the passed HTable, with the option of allowing + * non-transactional operations. + * @param hTable underlying HBase table to use + * @param conflictLevel level of conflict detection to perform (defaults to {@code COLUMN}) + * @param allowNonTransactional if true, additional operations (checkAndPut, increment, + * checkAndDelete) will be available, though non-transactional + */ + public TransactionAwareHTable(Table hTable, TxConstants.ConflictDetection conflictLevel, + boolean allowNonTransactional) { + super(conflictLevel, allowNonTransactional); + this.hTable = hTable; + } + + /* AbstractTransactionAwareTable implementation */ + + @Override + protected byte[] getTableKey() { + return hTable.getName().getName(); + } + + @Override + protected boolean doCommit() throws IOException { + return true; + } + + @Override + protected boolean doRollback() throws Exception { + try { + // pre-size arraylist of deletes + int size = 0; + for (Set cs : changeSets.values()) { + size += cs.size(); + } + List rollbackDeletes = new ArrayList<>(size); + for (Map.Entry> entry : changeSets.entrySet()) { + long transactionTimestamp = entry.getKey(); + for (ActionChange change : entry.getValue()) { + byte[] row = change.getRow(); + byte[] family = change.getFamily(); + byte[] qualifier = change.getQualifier(); + Delete rollbackDelete = new Delete(row); + makeRollbackOperation(rollbackDelete); + switch (conflictLevel) { + case ROW: + case NONE: + // issue family delete for the tx write pointer + rollbackDelete.addFamilyVersion(change.getFamily(), transactionTimestamp); + break; + case COLUMN: + if (family != null && qualifier == null) { + rollbackDelete.addFamilyVersion(family, transactionTimestamp); + } else if (family != null && qualifier != null) { + rollbackDelete.addColumn(family, qualifier, transactionTimestamp); + } + break; + default: + throw new IllegalStateException("Unknown conflict detection level: " + conflictLevel); + } + rollbackDeletes.add(rollbackDelete); + } + } + hTable.delete(rollbackDeletes); + return true; + } finally { + tx = null; + changeSets.clear(); + } + } + + /* HTableInterface implementation */ + + @Override + public TableName getName() { + return hTable.getName(); + } + + @Override + public Configuration getConfiguration() { + return hTable.getConfiguration(); + } + + @Override + public HTableDescriptor getTableDescriptor() throws IOException { + return hTable.getTableDescriptor(); + } + + @Override + public boolean exists(Get get) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + return hTable.exists(transactionalizeAction(get)); + } + + @Override + public void batch(List actions, Object[] results) + throws IOException, InterruptedException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + hTable.batch(transactionalizeActions(actions), results); + } + + @Override + public void batchCallback(List actions, Object[] results, + Batch.Callback callback) throws IOException, InterruptedException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + hTable.batchCallback(transactionalizeActions(actions), results, callback); + } + + @Override + public Result get(Get get) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + return hTable.get(transactionalizeAction(get)); + } + + @Override + public Result[] get(List gets) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + ArrayList transactionalizedGets = new ArrayList<>(); + for (Get get : gets) { + transactionalizedGets.add(transactionalizeAction(get)); + } + return hTable.get(transactionalizedGets); + } + + @Override + public ResultScanner getScanner(Scan scan) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + return hTable.getScanner(transactionalizeAction(scan)); + } + + @Override + public ResultScanner getScanner(byte[] family) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + Scan scan = new Scan(); + scan.addFamily(family); + return hTable.getScanner(transactionalizeAction(scan)); + } + + @Override + public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + Scan scan = new Scan(); + scan.addColumn(family, qualifier); + return hTable.getScanner(transactionalizeAction(scan)); + } + + @Override + public void put(Put put) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + Put txPut = transactionalizeAction(put); + hTable.put(txPut); + } + + @Override + public void put(List puts) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + List transactionalizedPuts = new ArrayList<>(puts.size()); + for (Put put : puts) { + Put txPut = transactionalizeAction(put); + transactionalizedPuts.add(txPut); + } + hTable.put(transactionalizedPuts); + } + + @Override + public void delete(Delete delete) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + hTable.delete(transactionalizeAction(delete)); + } + + @Override + public void delete(List deletes) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + List transactionalizedDeletes = new ArrayList<>(deletes.size()); + for (Delete delete : deletes) { + Delete txDelete = transactionalizeAction(delete); + transactionalizedDeletes.add(txDelete); + } + hTable.delete(transactionalizedDeletes); + } + + @Override + public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put) + throws IOException { + if (allowNonTransactional) { + return hTable.checkAndPut(row, family, qualifier, value, put); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value, + Delete delete) throws IOException { + if (allowNonTransactional) { + return hTable.checkAndDelete(row, family, qualifier, value, delete); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public boolean checkAndDelete(byte[] bytes, byte[] bytes1, byte[] bytes2, + CompareFilter.CompareOp compareOp, byte[] bytes3, Delete delete) throws IOException { + if (allowNonTransactional) { + return hTable.checkAndDelete(bytes, bytes1, bytes2, compareOp, bytes3, delete); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public boolean checkAndPut(byte[] bytes, byte[] bytes1, byte[] bytes2, + CompareFilter.CompareOp compareOp, byte[] bytes3, Put put) throws IOException { + if (allowNonTransactional) { + return hTable.checkAndPut(bytes, bytes1, bytes2, compareOp, bytes3, put); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, + CompareFilter.CompareOp compareOp, byte[] value, RowMutations rowMutations) + throws IOException { + if (allowNonTransactional) { + return hTable.checkAndMutate(row, family, qualifier, compareOp, value, rowMutations); + } + + throw new UnsupportedOperationException( + "checkAndMutate operation is not supported transactionally"); + } + + @Override + public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, + byte[] value, Put put) throws IOException { + if (allowNonTransactional) { + return hTable.checkAndPut(row, family, qualifier, value, put); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + + } + + @Override + public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, + byte[] value, Delete delete) throws IOException { + if (allowNonTransactional) { + return hTable.checkAndDelete(row, family, qualifier, op, value, delete); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, + byte[] value, RowMutations mutation) throws IOException { + if (allowNonTransactional) { + return hTable.checkAndMutate(row, family, qualifier, op, value, mutation); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public boolean[] existsAll(List gets) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + List transactionalizedGets = new ArrayList<>(gets.size()); + for (Get get : gets) { + transactionalizedGets.add(transactionalizeAction(get)); + } + return hTable.existsAll(transactionalizedGets); + } + + @Override + public void mutateRow(RowMutations rm) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + RowMutations transactionalMutations = new RowMutations(rm.getRow()); + for (Mutation mutation : rm.getMutations()) { + if (mutation instanceof Put) { + transactionalMutations.add(transactionalizeAction((Put) mutation)); + } else if (mutation instanceof Delete) { + transactionalMutations.add(transactionalizeAction((Delete) mutation)); + } + } + hTable.mutateRow(transactionalMutations); + } + + @Override + public Result append(Append append) throws IOException { + if (allowNonTransactional) { + return hTable.append(append); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public Result increment(Increment increment) throws IOException { + if (allowNonTransactional) { + return hTable.increment(increment); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) + throws IOException { + if (allowNonTransactional) { + return hTable.incrementColumnValue(row, family, qualifier, amount); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, + Durability durability) throws IOException { + if (allowNonTransactional) { + return hTable.incrementColumnValue(row, family, qualifier, amount, durability); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + @Override + public void close() throws IOException { + hTable.close(); + } + + @Override + public CoprocessorRpcChannel coprocessorService(byte[] row) { + return hTable.coprocessorService(row); + } + + @Override + public Map coprocessorService(Class service, byte[] startKey, + byte[] endKey, Batch.Call callable) throws ServiceException, Throwable { + return hTable.coprocessorService(service, startKey, endKey, callable); + } + + @Override + public void coprocessorService(Class service, byte[] startKey, + byte[] endKey, Batch.Call callable, Batch.Callback callback) + throws ServiceException, Throwable { + hTable.coprocessorService(service, startKey, endKey, callable, callback); + } + + @Override + public Map batchCoprocessorService( + MethodDescriptor methodDescriptor, Message request, byte[] startKey, byte[] endKey, + R responsePrototype) throws ServiceException, Throwable { + return hTable.batchCoprocessorService(methodDescriptor, request, startKey, endKey, + responsePrototype); + } + + @Override + public void batchCoprocessorService(MethodDescriptor methodDescriptor, + Message request, byte[] startKey, byte[] endKey, R responsePrototype, Callback callback) + throws ServiceException, Throwable { + hTable.batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype, + callback); + } + + public void addToOperation(OperationWithAttributes op, Transaction tx) throws IOException { + op.setAttribute(TxConstants.TX_OPERATION_ATTRIBUTE_KEY, txCodec.encode(tx)); + } + + protected void makeRollbackOperation(Delete delete) { + delete.setAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY, new byte[0]); + } + + @Override + public TableDescriptor getDescriptor() throws IOException { + return hTable.getDescriptor(); + } + + @Override + public boolean[] exists(List gets) throws IOException { + if (tx == null) { + throw new IOException("Transaction not started"); + } + List transactionalizedGets = new ArrayList<>(gets.size()); + for (Get get : gets) { + transactionalizedGets.add(transactionalizeAction(get)); + } + return hTable.exists(transactionalizedGets); + } + + @Override + public long getRpcTimeout(TimeUnit unit) { + return hTable.getRpcTimeout(unit); + } + + @Override + public int getRpcTimeout() { + return hTable.getRpcTimeout(); + } + + @Override + public void setRpcTimeout(int rpcTimeout) { + hTable.setRpcTimeout(rpcTimeout); + + } + + @Override + public long getReadRpcTimeout(TimeUnit unit) { + return hTable.getReadRpcTimeout(unit); + } + + @Override + public int getReadRpcTimeout() { + return hTable.getReadRpcTimeout(); + } + + @Override + public void setReadRpcTimeout(int readRpcTimeout) { + hTable.setReadRpcTimeout(readRpcTimeout); + + } + + @Override + public long getWriteRpcTimeout(TimeUnit unit) { + return hTable.getWriteRpcTimeout(unit); + } + + @Override + public int getWriteRpcTimeout() { + return hTable.getWriteRpcTimeout(); + } + + @Override + public void setWriteRpcTimeout(int writeRpcTimeout) { + hTable.setWriteRpcTimeout(writeRpcTimeout); + + } + + @Override + public long getOperationTimeout(TimeUnit unit) { + return hTable.getOperationTimeout(unit); + } + + @Override + public int getOperationTimeout() { + return hTable.getOperationTimeout(); + } + + @Override + public void setOperationTimeout(int operationTimeout) { + hTable.setOperationTimeout(operationTimeout); + ; + } + + @Override + public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) { + if (allowNonTransactional) { + return hTable.checkAndMutate(row, family); + } else { + throw new UnsupportedOperationException("Operation is not supported transactionally"); + } + } + + // Helpers to get copies of objects with the timestamp set to the current transaction timestamp. + + private Get transactionalizeAction(Get get) throws IOException { + addToOperation(get, tx); + return get; + } + + private Scan transactionalizeAction(Scan scan) throws IOException { + addToOperation(scan, tx); + return scan; + } + + private Put transactionalizeAction(Put put) throws IOException { + Put txPut = new Put(put.getRow(), tx.getWritePointer()); + Set>> familyMap = put.getFamilyCellMap().entrySet(); + if (!familyMap.isEmpty()) { + for (Map.Entry> family : familyMap) { + List familyValues = family.getValue(); + if (!familyValues.isEmpty()) { + for (Cell value : familyValues) { + txPut.addColumn(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value), + tx.getWritePointer(), CellUtil.cloneValue(value)); + addToChangeSet(txPut.getRow(), CellUtil.cloneFamily(value), + CellUtil.cloneQualifier(value)); + } + } + } + } + for (Map.Entry entry : put.getAttributesMap().entrySet()) { + txPut.setAttribute(entry.getKey(), entry.getValue()); + } + txPut.setDurability(put.getDurability()); + addToOperation(txPut, tx); + return txPut; + } + + private Delete transactionalizeAction(Delete delete) throws IOException { + long transactionTimestamp = tx.getWritePointer(); + + byte[] deleteRow = delete.getRow(); + Delete txDelete = new Delete(deleteRow, transactionTimestamp); + + Map> familyToDelete = delete.getFamilyCellMap(); + if (familyToDelete.isEmpty()) { + // perform a row delete if we are using row-level conflict detection + if (conflictLevel == TxConstants.ConflictDetection.ROW + || conflictLevel == TxConstants.ConflictDetection.NONE) { + // Row delete leaves delete markers in all column families of the table + // Therefore get all the column families of the hTable from the HTableDescriptor and + // add them to the + // changeSet + for (HColumnDescriptor columnDescriptor : hTable.getTableDescriptor().getColumnFamilies()) { + // no need to identify individual columns deleted + addToChangeSet(deleteRow, columnDescriptor.getName(), null); + } + } else { + Result result = get(new Get(delete.getRow())); + // Delete everything + NavigableMap> resultMap = result.getNoVersionMap(); + for (Map.Entry> familyEntry : resultMap.entrySet()) { + NavigableMap familyColumns = result.getFamilyMap(familyEntry.getKey()); + for (Map.Entry column : familyColumns.entrySet()) { + txDelete.addColumns(familyEntry.getKey(), column.getKey(), transactionTimestamp); + addToChangeSet(deleteRow, familyEntry.getKey(), column.getKey()); + } + } + } + } else { + for (Map.Entry> familyEntry : familyToDelete.entrySet()) { + byte[] family = familyEntry.getKey(); + List entries = familyEntry.getValue(); + boolean isFamilyDelete = false; + if (entries.size() == 1) { + Cell cell = entries.get(0); + isFamilyDelete = CellUtil.isDeleteFamily(cell); + } + if (isFamilyDelete) { + if (conflictLevel == TxConstants.ConflictDetection.ROW + || conflictLevel == TxConstants.ConflictDetection.NONE) { + // no need to identify individual columns deleted + txDelete.addFamily(family); + addToChangeSet(deleteRow, family, null); + } else { + Result result = get(new Get(delete.getRow()).addFamily(family)); + // Delete entire family + NavigableMap familyColumns = result.getFamilyMap(family); + for (Map.Entry column : familyColumns.entrySet()) { + txDelete.addColumns(family, column.getKey(), transactionTimestamp); + addToChangeSet(deleteRow, family, column.getKey()); + } + } + } else { + for (Cell value : entries) { + txDelete.addColumn(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value), + transactionTimestamp); + addToChangeSet(deleteRow, CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value)); + } + } + } + } + for (Map.Entry entry : delete.getAttributesMap().entrySet()) { + txDelete.setAttribute(entry.getKey(), entry.getValue()); + } + txDelete.setDurability(delete.getDurability()); + addToOperation(txDelete, tx); + return txDelete; + } + + private List transactionalizeActions(List actions) + throws IOException { + List transactionalizedActions = new ArrayList<>(actions.size()); + for (Row action : actions) { + if (action instanceof Get) { + transactionalizedActions.add(transactionalizeAction((Get) action)); + } else if (action instanceof Put) { + transactionalizedActions.add(transactionalizeAction((Put) action)); + } else if (action instanceof Delete) { + transactionalizedActions.add(transactionalizeAction((Delete) action)); + } else { + transactionalizedActions.add(action); + } + } + return transactionalizedActions; + } + +} + diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java new file mode 100644 index 00000000..4dbe924a --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/CellSkipFilter.java @@ -0,0 +1,129 @@ +/* + * 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.tephra.hbase.coprocessor; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterBase; + +import java.io.IOException; +import java.util.List; + +/** + * {@link Filter} that encapsulates another {@link Filter}. It remembers the last {@link KeyValue} + * for which the underlying filter returned the {@link ReturnCode#NEXT_COL} or {@link ReturnCode#INCLUDE_AND_NEXT_COL}, + * so that when {@link #filterKeyValue} is called again for the same {@link KeyValue} with different + * version, it returns {@link ReturnCode#NEXT_COL} directly without consulting the underlying {@link Filter}. + * Please see TEPHRA-169 for more details. + */ +public class CellSkipFilter extends FilterBase { + private final Filter filter; + // remember the previous keyvalue processed by filter when the return code was NEXT_COL or INCLUDE_AND_NEXT_COL + private KeyValue skipColumn = null; + + public CellSkipFilter(Filter filter) { + this.filter = filter; + } + + /** + * Determines whether the current cell should be skipped. The cell will be skipped + * if the previous keyvalue had the same key as the current cell. This means filter already responded + * for the previous keyvalue with ReturnCode.NEXT_COL or ReturnCode.INCLUDE_AND_NEXT_COL. + * @param cell the {@link Cell} to be tested for skipping + * @return true is current cell should be skipped, false otherwise + */ + private boolean skipCellVersion(Cell cell) { + return skipColumn != null + && CellUtil.matchingRow(cell, skipColumn) + && CellUtil.matchingFamily(cell, skipColumn) + && CellUtil.matchingQualifier(cell, skipColumn); + } + + @Override + public ReturnCode filterKeyValue(Cell cell) throws IOException { + if (skipCellVersion(cell)) { + return ReturnCode.NEXT_COL; + } + + ReturnCode code = filter.filterKeyValue(cell); + if (code == ReturnCode.NEXT_COL || code == ReturnCode.INCLUDE_AND_NEXT_COL) { + // only store the reference to the keyvalue if we are returning NEXT_COL or INCLUDE_AND_NEXT_COL + skipColumn = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), + cell.getFamilyArray(), cell.getFamilyOffset(), + cell.getFamilyLength(), cell.getQualifierArray(), + cell.getQualifierOffset(), cell.getQualifierLength()); + } else { + skipColumn = null; + } + return code; + } + + @Override + public boolean filterRow() throws IOException { + return filter.filterRow(); + } + + @Override + public Cell transformCell(Cell cell) throws IOException { + return filter.transformCell(cell); + } + + @Override + public void reset() throws IOException { + filter.reset(); + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { + return filter.filterRowKey(buffer, offset, length); + } + + @Override + public boolean filterAllRemaining() throws IOException { + return filter.filterAllRemaining(); + } + + @Override + public void filterRowCells(List kvs) throws IOException { + filter.filterRowCells(kvs); + } + + @Override + public boolean hasFilterRow() { + return filter.hasFilterRow(); + } + + @Override + public Cell getNextCellHint(Cell currentKV) throws IOException { + return filter.getNextCellHint(currentKV); + } + + @Override + public boolean isFamilyEssential(byte[] name) throws IOException { + return filter.isFamilyEssential(name); + } + + @Override + public byte[] toByteArray() throws IOException { + return filter.toByteArray(); + } +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/FilteredInternalScanner.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/FilteredInternalScanner.java new file mode 100644 index 00000000..f81487ef --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/FilteredInternalScanner.java @@ -0,0 +1,82 @@ +/* + * 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.tephra.hbase.coprocessor; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.Filter.ReturnCode; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.ScannerContext; +import org.apache.tephra.hbase.coprocessor.TransactionProcessor.IncludeInProgressFilter; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Wrapper of InternalScanner to apply Transaction visibility filter for flush and compact + */ +public class FilteredInternalScanner implements InternalScanner { + + private final InternalScanner delegate; + private final Filter filter; + private List outResult = new ArrayList(); + + public FilteredInternalScanner(InternalScanner internalScanner, IncludeInProgressFilter filter) { + this.delegate = internalScanner; + this.filter = filter; + } + + @Override + public void close() throws IOException { + this.delegate.close(); + } + + @Override + public boolean next(List result, ScannerContext scannerContext) throws IOException { + outResult.clear(); + if (filter.filterAllRemaining()) { return false; } + while (true) { + boolean next = delegate.next(outResult, scannerContext); + for (Cell cell : outResult) { + ReturnCode code = filter.filterKeyValue(cell); + switch (code) { + // included, so we are done + case INCLUDE: + case INCLUDE_AND_NEXT_COL: + result.add(cell); + break; + case SKIP: + case NEXT_COL: + case NEXT_ROW: + default: + break; + } + } + if (!next) { + return next; + } + if (!result.isEmpty()) { + return true; + } + + } + } + +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java new file mode 100644 index 00000000..0ca9f9c2 --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionFilters.java @@ -0,0 +1,62 @@ +/* + * 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.tephra.hbase.coprocessor; + +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.tephra.Transaction; + +import java.util.Map; +import javax.annotation.Nullable; + +/** + * Factory class for providing {@link Filter} instances. + */ +public class TransactionFilters { + /** + * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. + * + * @param tx the current transaction to apply. Only data visible to this transaction will be returned. + * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name + * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} + * these will be interpreted as "delete" markers and the column will be filtered out + * @param scanType the type of scan operation being performed + */ + public static Filter getVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, + ScanType scanType) { + return new CellSkipFilter(new TransactionVisibilityFilter(tx, ttlByFamily, allowEmptyValues, scanType, null)); + } + + /** + * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. + * + * @param tx the current transaction to apply. Only data visible to this transaction will be returned. + * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name + * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} + * these will be interpreted as "delete" markers and the column will be filtered out + * @param scanType the type of scan operation being performed + * @param cellFilter if non-null, this filter will be applied to all cells visible to the current transaction, by + * calling {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)}. If null, then + * {@link Filter.ReturnCode#INCLUDE_AND_NEXT_COL} will be returned instead. + */ + public static Filter getVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, + ScanType scanType, @Nullable Filter cellFilter) { + return new CellSkipFilter(new TransactionVisibilityFilter(tx, ttlByFamily, allowEmptyValues, scanType, cellFilter)); + } +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java new file mode 100644 index 00000000..4b7a516d --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionProcessor.java @@ -0,0 +1,574 @@ +/* + * 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.tephra.hbase.coprocessor; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.OperationWithAttributes; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.ScanOptions; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.tephra.Transaction; +import org.apache.tephra.TransactionCodec; +import org.apache.tephra.TxConstants; +import org.apache.tephra.coprocessor.CacheSupplier; +import org.apache.tephra.coprocessor.TransactionStateCache; +import org.apache.tephra.coprocessor.TransactionStateCacheSupplier; +import org.apache.tephra.hbase.txprune.CompactionState; +import org.apache.tephra.persist.TransactionVisibilityState; +import org.apache.tephra.util.TxUtils; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import javax.annotation.Nullable; + +/** + * {@code org.apache.hadoop.hbase.coprocessor.RegionObserver} coprocessor that handles server-side processing + * for transactions: + *
    + *
  • applies filtering to exclude data from invalid and in-progress transactions
  • + *
  • overrides the scanner returned for flush and compaction to drop data written by invalidated transactions, + * or expired due to TTL.
  • + *
+ * + *

In order to use this coprocessor for transactions, configure the class on any table involved in transactions, + * or on all user tables by adding the following to hbase-site.xml: + * {@code + * + * hbase.coprocessor.region.classes + * org.apache.tephra.hbase.coprocessor.TransactionProcessor + * + * } + *

+ * + *

HBase {@code Get} and {@code Scan} operations should have the current transaction serialized on to the operation + * as an attribute: + * {@code + * Transaction t = ...; + * Get get = new Get(...); + * TransactionCodec codec = new TransactionCodec(); + * codec.addToOperation(get, t); + * } + *

+ */ +public class TransactionProcessor implements RegionObserver, RegionCoprocessor { + private static final Log LOG = LogFactory.getLog(TransactionProcessor.class); + + private final TransactionCodec txCodec; + private TransactionStateCache cache; + private volatile CompactionState compactionState; + private CacheSupplier cacheSupplier; + + protected volatile Boolean pruneEnable; + protected volatile Long txMaxLifetimeMillis; + protected Map ttlByFamily = Maps.newTreeMap(Bytes.BYTES_COMPARATOR); + protected boolean allowEmptyValues = TxConstants.ALLOW_EMPTY_VALUES_DEFAULT; + protected boolean readNonTxnData = TxConstants.DEFAULT_READ_NON_TX_DATA; + + public TransactionProcessor() { + this.txCodec = new TransactionCodec(); + } + + @Override + public Optional getRegionObserver() { + return Optional.of(this); + } + + /* RegionObserver implementation */ + + @Override + public void start(CoprocessorEnvironment e) throws IOException { + if (e instanceof RegionCoprocessorEnvironment) { + RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e; + this.cacheSupplier = getTransactionStateCacheSupplier(env); + this.cache = cacheSupplier.get(); + + TableDescriptor tableDesc = env.getRegion().getTableDescriptor(); + for (ColumnFamilyDescriptor columnDesc : tableDesc.getColumnFamilies()) { + byte[] columnTTL = columnDesc.getValue(Bytes.toBytes(TxConstants.PROPERTY_TTL)); + long ttl = 0; + if (columnTTL != null) { + try { + ttl = Long.parseLong(Bytes.toString(columnTTL)); + LOG.info("Family " + columnDesc.getNameAsString() + " has TTL of " + ttl); + } catch (NumberFormatException nfe) { + LOG.warn("Invalid TTL value configured for column family " + + columnDesc.getNameAsString() + ", value = " + Bytes.toString(columnTTL)); + } + } + ttlByFamily.put(columnDesc.getName(), ttl); + } + + this.allowEmptyValues = getAllowEmptyValues(env, tableDesc); + this.txMaxLifetimeMillis = getTxMaxLifetimeMillis(env); + this.readNonTxnData = Boolean.valueOf(tableDesc.getValue(TxConstants.READ_NON_TX_DATA)); + if (readNonTxnData) { + LOG.info("Reading pre-existing data enabled for table " + + tableDesc.getTableName().getNameAsString()); + } + initializePruneState(env); + } + } + + /** + * Fetch the {@link Configuration} that contains the properties required by the coprocessor. By + * default, the HBase configuration is returned. This method will never return {@code null} in + * Tephra but the derived classes might do so if {@link Configuration} is not available + * temporarily (for example, if it is being fetched from a HBase Table. + * @param env {@link RegionCoprocessorEnvironment} of the Region to which the coprocessor is + * associated + * @return {@link Configuration}, can be null if it is not available + */ + @Nullable + protected Configuration getConfiguration(CoprocessorEnvironment env) { + return env.getConfiguration(); + } + + protected CacheSupplier + getTransactionStateCacheSupplier(RegionCoprocessorEnvironment env) { + return new TransactionStateCacheSupplier(env.getConfiguration()); + } + + @Override + public void stop(CoprocessorEnvironment e) throws IOException { + try { + resetPruneState(); + } finally { + if (cacheSupplier != null) { + cacheSupplier.release(); + } + } + } + + @Override + public void preGetOp(ObserverContext e, Get get, List results) + throws IOException { + Transaction tx = getFromOperation(get); + if (tx != null) { + projectFamilyDeletes(get); + get.setMaxVersions(); + get.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx, readNonTxnData), + TxUtils.getMaxVisibleTimestamp(tx)); + Filter newFilter = getTransactionFilter(tx, ScanType.USER_SCAN, get.getFilter()); + get.setFilter(newFilter); + } + } + + @Override + public void prePut(ObserverContext e, Put put, WALEdit edit, + Durability durability) throws IOException { + Transaction tx = getFromOperation(put); + ensureValidTxLifetime(e.getEnvironment(), put, tx); + } + + @Override + public void preDelete(ObserverContext e, Delete delete, + WALEdit edit, Durability durability) throws IOException { + // Translate deletes into our own delete tombstones + // Since HBase deletes cannot be undone, we need to translate deletes into special puts, + // which allows + // us to rollback the changes (by a real delete) if the transaction fails + + // Deletes that are part of a transaction rollback do not need special handling. + // They will never be rolled back, so are performed as normal HBase deletes. + if (isRollbackOperation(delete)) { + return; + } + + Transaction tx = getFromOperation(delete); + ensureValidTxLifetime(e.getEnvironment(), delete, tx); + + // Other deletes are client-initiated and need to be translated into our own tombstones + // TODO: this should delegate to the DeleteStrategy implementation. + Put deleteMarkers = new Put(delete.getRow(), delete.getTimeStamp()); + for (byte[] family : delete.getFamilyCellMap().keySet()) { + List familyCells = delete.getFamilyCellMap().get(family); + if (isFamilyDelete(familyCells)) { + deleteMarkers.addColumn(family, TxConstants.FAMILY_DELETE_QUALIFIER, + familyCells.get(0).getTimestamp(), HConstants.EMPTY_BYTE_ARRAY); + } else { + for (Cell cell : familyCells) { + deleteMarkers.addColumn(family, CellUtil.cloneQualifier(cell), cell.getTimestamp(), + HConstants.EMPTY_BYTE_ARRAY); + } + } + } + for (Map.Entry entry : delete.getAttributesMap().entrySet()) { + deleteMarkers.setAttribute(entry.getKey(), entry.getValue()); + } + e.getEnvironment().getRegion().put(deleteMarkers); + // skip normal delete handling + e.bypass(); + } + + private boolean getAllowEmptyValues(RegionCoprocessorEnvironment env, TableDescriptor htd) { + String allowEmptyValuesFromTableDesc = htd.getValue(TxConstants.ALLOW_EMPTY_VALUES_KEY); + Configuration conf = getConfiguration(env); + boolean allowEmptyValuesFromConfig = + (conf != null) ? conf.getBoolean(TxConstants.ALLOW_EMPTY_VALUES_KEY, + TxConstants.ALLOW_EMPTY_VALUES_DEFAULT) : TxConstants.ALLOW_EMPTY_VALUES_DEFAULT; + + // If the property is not present in the tableDescriptor, get it from the Configuration + return (allowEmptyValuesFromTableDesc != null) ? Boolean.valueOf(allowEmptyValuesFromTableDesc) + : allowEmptyValuesFromConfig; + } + + private long getTxMaxLifetimeMillis(RegionCoprocessorEnvironment env) { + Configuration conf = getConfiguration(env); + if (conf != null) { + return TimeUnit.SECONDS.toMillis(conf.getInt(TxConstants.Manager.CFG_TX_MAX_LIFETIME, + TxConstants.Manager.DEFAULT_TX_MAX_LIFETIME)); + } + return TimeUnit.SECONDS.toMillis(TxConstants.Manager.DEFAULT_TX_MAX_LIFETIME); + } + + private boolean isFamilyDelete(List familyCells) { + return familyCells.size() == 1 && CellUtil.isDeleteFamily(familyCells.get(0)); + } + + @Override + public void preScannerOpen( + org.apache.hadoop.hbase.coprocessor.ObserverContext c, + Scan scan) throws IOException { + Transaction tx = getFromOperation(scan); + if (tx != null) { + projectFamilyDeletes(scan); + scan.setMaxVersions(); + scan.setTimeRange(TxUtils.getOldestVisibleTimestamp(ttlByFamily, tx, readNonTxnData), + TxUtils.getMaxVisibleTimestamp(tx)); + Filter newFilter = getTransactionFilter(tx, ScanType.USER_SCAN, scan.getFilter()); + scan.setFilter(newFilter); + } + } + + /** + * Ensures that family delete markers are present in the columns requested for any scan operation. + * @param scan The original scan request + * @return The modified scan request with the family delete qualifiers represented + */ + private Scan projectFamilyDeletes(Scan scan) { + for (Map.Entry> entry : scan.getFamilyMap().entrySet()) { + NavigableSet columns = entry.getValue(); + // wildcard scans will automatically include the delete marker, so only need to add it + // when we have + // explicit columns listed + if (columns != null && !columns.isEmpty()) { + scan.addColumn(entry.getKey(), TxConstants.FAMILY_DELETE_QUALIFIER); + } + } + return scan; + } + + /** + * Ensures that family delete markers are present in the columns requested for any get operation. + * @param get The original get request + * @return The modified get request with the family delete qualifiers represented + */ + private Get projectFamilyDeletes(Get get) { + for (Map.Entry> entry : get.getFamilyMap().entrySet()) { + NavigableSet columns = entry.getValue(); + // wildcard scans will automatically include the delete marker, so only need to add it + // when we have + // explicit columns listed + if (columns != null && !columns.isEmpty()) { + get.addColumn(entry.getKey(), TxConstants.FAMILY_DELETE_QUALIFIER); + } + } + return get; + } + + @Override + public void preFlushScannerOpen( + org.apache.hadoop.hbase.coprocessor.ObserverContext c, + Store store, org.apache.hadoop.hbase.regionserver.ScanOptions options, + org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker tracker) throws IOException { + if (cache.getLatestState() != null) { + options.readAllVersions(); + } + } + + public InternalScanner preFlush( + org.apache.hadoop.hbase.coprocessor.ObserverContext c, + Store store, InternalScanner scanner, + org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker tracker) throws IOException { + InternalScanner s = + createStoreScanner(c.getEnvironment(), "flush", cache.getLatestState(), scanner, + ScanType.COMPACT_RETAIN_DELETES); + if (s != null) { + return s; + } + return scanner; + } + + @Override + public void postFlush( + org.apache.hadoop.hbase.coprocessor.ObserverContext e, + org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker tracker) throws IOException { + // Record whether the region is empty after a flush + Region region = e.getEnvironment().getRegion(); + // After a flush, if the memstore size is zero and there are no store files for any stores + // in the region + // then the region must be empty + long numStoreFiles = numStoreFilesForRegion(e); + long memstoreSize = region.getMemStoreSize(); + LOG.debug(String.format("Region %s: memstore size = %s, num store files = %s", + region.getRegionInfo().getRegionNameAsString(), memstoreSize, numStoreFiles)); + if (memstoreSize == 0 && numStoreFiles == 0) { + if (compactionState != null) { + compactionState.persistRegionEmpty(System.currentTimeMillis()); + } + } + } + + @Override + public void preCompactScannerOpen(ObserverContext c, Store store, + ScanType scanType, ScanOptions options, CompactionLifeCycleTracker tracker, + CompactionRequest request) throws IOException { + if (cache.getLatestState() != null) { + options.readAllVersions(); + } + } + + @Override + public InternalScanner preCompact( + org.apache.hadoop.hbase.coprocessor.ObserverContext c, + Store store, InternalScanner scanner, ScanType scanType, + org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker tracker, + CompactionRequest request) throws IOException { + // Get the latest tx snapshot state for the compaction + TransactionVisibilityState snapshot = cache.getLatestState(); + // Record tx state before the compaction + if (compactionState != null) { + compactionState.record(request, snapshot); + } + // Also make sure to use the same snapshot for the compaction + InternalScanner s = + createStoreScanner(c.getEnvironment(), "compaction", snapshot, scanner, scanType); + if (s != null) { + return s; + } + return scanner; + } + + @Override + public void postCompact( + org.apache.hadoop.hbase.coprocessor.ObserverContext c, + Store store, StoreFile resultFile, + org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker tracker, + CompactionRequest request) throws IOException { + // Persist the compaction state after a successful compaction + if (compactionState != null) { + compactionState.persist(); + } + } + + protected InternalScanner createStoreScanner(RegionCoprocessorEnvironment env, String action, + TransactionVisibilityState snapshot, InternalScanner scanner, + ScanType type) throws IOException { + if (snapshot == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Region " + env.getRegion().getRegionInfo().getRegionNameAsString() + + ", no current transaction state found, defaulting to normal " + action + " scanner"); + } + return null; + } + // construct a dummy transaction from the latest snapshot + Transaction dummyTx = TxUtils.createDummyTransaction(snapshot); + return new FilteredInternalScanner(scanner, + new IncludeInProgressFilter(dummyTx.getVisibilityUpperBound(), snapshot.getInvalid(), + getTransactionFilter(dummyTx, type, null))); + } + + private Transaction getFromOperation(OperationWithAttributes op) throws IOException { + byte[] encoded = op.getAttribute(TxConstants.TX_OPERATION_ATTRIBUTE_KEY); + if (encoded == null) { + // to support old clients + encoded = op.getAttribute(TxConstants.OLD_TX_OPERATION_ATTRIBUTE_KEY); + } + if (encoded != null) { + return txCodec.decode(encoded); + } + return null; + } + + /** + * Make sure that the transaction is within the max valid transaction lifetime. + * @param env {@link RegionCoprocessorEnvironment} of the Region to which the coprocessor is + * associated + * @param op {@link OperationWithAttributes} HBase operation to access its attributes if required + * @param tx {@link Transaction} supplied by the + * @throws DoNotRetryIOException thrown if the transaction is older than the max lifetime of a + * transaction IOException throw if the value of max lifetime of transaction is + * unavailable + */ + protected void ensureValidTxLifetime(RegionCoprocessorEnvironment env, + @SuppressWarnings("unused") OperationWithAttributes op, @Nullable Transaction tx) + throws IOException { + if (tx == null) { + return; + } + + boolean validLifetime = + (TxUtils.getTimestamp(tx.getTransactionId()) + txMaxLifetimeMillis) > System + .currentTimeMillis(); + if (!validLifetime) { + throw new DoNotRetryIOException( + String.format("Transaction %s has exceeded max lifetime %s ms", tx.getTransactionId(), + txMaxLifetimeMillis)); + } + } + + private boolean isRollbackOperation(OperationWithAttributes op) throws IOException { + return op.getAttribute(TxConstants.TX_ROLLBACK_ATTRIBUTE_KEY) != null || + // to support old clients + op.getAttribute(TxConstants.OLD_TX_ROLLBACK_ATTRIBUTE_KEY) != null; + } + + /** + * Derived classes can override this method to customize the filter used to return data visible + * for the current transaction. + * @param tx the current transaction to apply + * @param type the type of scan being performed + */ + protected Filter getTransactionFilter(Transaction tx, ScanType type, Filter filter) { + return TransactionFilters.getVisibilityFilter(tx, ttlByFamily, allowEmptyValues, type, filter); + } + + /** + * Refresh the properties related to transaction pruning. This method needs to be invoked if there + * is change in the prune related properties after clearing the state by calling + * {@link #resetPruneState}. + * @param env {@link RegionCoprocessorEnvironment} of this region + */ + protected void initializePruneState(RegionCoprocessorEnvironment env) { + Configuration conf = getConfiguration(env); + if (conf != null) { + pruneEnable = + conf.getBoolean(TxConstants.TransactionPruning.PRUNE_ENABLE, + TxConstants.TransactionPruning.DEFAULT_PRUNE_ENABLE); + + if (Boolean.TRUE.equals(pruneEnable)) { + TableName pruneTable = + TableName.valueOf(conf.get(TxConstants.TransactionPruning.PRUNE_STATE_TABLE, + TxConstants.TransactionPruning.DEFAULT_PRUNE_STATE_TABLE)); + long pruneFlushInterval = + TimeUnit.SECONDS + .toMillis(conf.getLong(TxConstants.TransactionPruning.PRUNE_FLUSH_INTERVAL, + TxConstants.TransactionPruning.DEFAULT_PRUNE_FLUSH_INTERVAL)); + + compactionState = new CompactionState(env, pruneTable, pruneFlushInterval); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format( + "Automatic invalid list pruning is enabled for table %s. Compaction state " + + "will be recorded in table %s", + env.getRegionInfo().getTable().getNameWithNamespaceInclAsString(), + pruneTable.getNameWithNamespaceInclAsString())); + } + } + } + } + + /** + * Stop and clear state related to pruning. + */ + protected void resetPruneState() { + pruneEnable = false; + if (compactionState != null) { + compactionState.stop(); + compactionState = null; + } + } + + private long numStoreFilesForRegion(ObserverContext c) { + long numStoreFiles = 0; + for (Store store : c.getEnvironment().getRegion().getStores()) { + numStoreFiles += store.getStorefiles().size(); + } + return numStoreFiles; + } + + /** + * Filter used to include cells visible to in-progress transactions on flush and commit. + */ + static class IncludeInProgressFilter extends FilterBase { + private final long visibilityUpperBound; + private final Set invalidIds; + private final Filter txFilter; + + public IncludeInProgressFilter(long upperBound, Collection invalids, + Filter transactionFilter) { + this.visibilityUpperBound = upperBound; + this.invalidIds = Sets.newHashSet(invalids); + this.txFilter = transactionFilter; + } + + @Override + public ReturnCode filterKeyValue(Cell cell) throws IOException { + // include all cells visible to in-progress transactions, except for those already + // marked as invalid + long ts = cell.getTimestamp(); + if (ts > visibilityUpperBound) { + // include everything that could still be in-progress except invalids + if (invalidIds.contains(ts)) { + return ReturnCode.SKIP; + } + return ReturnCode.INCLUDE; + } + return txFilter.filterKeyValue(cell); + } + } +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java new file mode 100644 index 00000000..b8fa587e --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/coprocessor/TransactionVisibilityFilter.java @@ -0,0 +1,305 @@ +/* + * 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.tephra.hbase.coprocessor; + +import com.google.common.collect.Maps; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.tephra.Transaction; +import org.apache.tephra.TxConstants; +import org.apache.tephra.util.TxUtils; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; + +/** + * Applies filtering of data based on transactional visibility (HBase 2.0 specific version). + * Note: this is intended for server-side use only, as additional properties need to be set on + * any {@code Scan} or {@code Get} operation performed. + */ +public class TransactionVisibilityFilter extends FilterBase { + private final Transaction tx; + // oldest visible timestamp by column family, used to apply TTL when reading + private final Map oldestTsByFamily; + // if false, empty values will be interpreted as deletes + private final boolean allowEmptyValues; + // whether or not we can remove delete markers + // these can only be safely removed when we are traversing all storefiles + private final boolean clearDeletes; + // optional sub-filter to apply to visible cells + private final Filter cellFilter; + // since we traverse KVs in order, cache the current oldest TS to avoid map lookups per KV + private final ImmutableBytesWritable currentFamily = new ImmutableBytesWritable(HConstants.EMPTY_BYTE_ARRAY); + + private long currentOldestTs; + + private DeleteTracker deleteTracker = new DeleteTracker(); + + /** + * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. + * + * @param tx the current transaction to apply. Only data visible to this transaction will be returned. + * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name + * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} + * these will be interpreted as "delete" markers and the column will be filtered out + * @param scanType the type of scan operation being performed + */ + public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, + ScanType scanType) { + this(tx, ttlByFamily, allowEmptyValues, scanType, null); + } + + /** + * Creates a new {@link org.apache.hadoop.hbase.filter.Filter} for returning data only from visible transactions. + * + * @param tx the current transaction to apply. Only data visible to this transaction will be returned. + * @param ttlByFamily map of time-to-live (TTL) (in milliseconds) by column family name + * @param allowEmptyValues if {@code true} cells with empty {@code byte[]} values will be returned, if {@code false} + * these will be interpreted as "delete" markers and the column will be filtered out + * @param scanType the type of scan operation being performed + * @param cellFilter if non-null, this filter will be applied to all cells visible to the current transaction, by + * calling {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)}. If null, then + * {@link Filter.ReturnCode#INCLUDE_AND_NEXT_COL} will be returned instead. + */ + public TransactionVisibilityFilter(Transaction tx, Map ttlByFamily, boolean allowEmptyValues, + ScanType scanType, @Nullable Filter cellFilter) { + this.tx = tx; + this.oldestTsByFamily = Maps.newTreeMap(); + for (Map.Entry ttlEntry : ttlByFamily.entrySet()) { + long familyTTL = ttlEntry.getValue(); + oldestTsByFamily.put(new ImmutableBytesWritable(ttlEntry.getKey()), + familyTTL <= 0 ? 0 : tx.getVisibilityUpperBound() - familyTTL * TxConstants.MAX_TX_PER_MS); + } + this.allowEmptyValues = allowEmptyValues; + this.clearDeletes = + scanType == ScanType.COMPACT_DROP_DELETES || + (scanType == ScanType.USER_SCAN && tx.getVisibilityLevel() != Transaction.VisibilityLevel.SNAPSHOT_ALL); + this.cellFilter = cellFilter; + } + + @Override + public ReturnCode filterKeyValue(Cell cell) throws IOException { + if (!CellUtil.matchingFamily(cell, currentFamily.get(), currentFamily.getOffset(), currentFamily.getLength())) { + // column family changed + currentFamily.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + Long familyOldestTs = oldestTsByFamily.get(currentFamily); + currentOldestTs = familyOldestTs != null ? familyOldestTs : 0; + deleteTracker.reset(); + } + // need to apply TTL for the column family here + long kvTimestamp = cell.getTimestamp(); + if (TxUtils.getTimestampForTTL(kvTimestamp) < currentOldestTs) { + // passed TTL for this column, seek to next + return ReturnCode.NEXT_COL; + } else if (tx.isVisible(kvTimestamp)) { + // Return all writes done by current transaction (including deletes) for VisibilityLevel.SNAPSHOT_ALL + if (tx.getVisibilityLevel() == Transaction.VisibilityLevel.SNAPSHOT_ALL && tx.isCurrentWrite(kvTimestamp)) { + // cell is visible + // visibility SNAPSHOT_ALL needs all matches + return runSubFilter(ReturnCode.INCLUDE, cell); + } + if (DeleteTracker.isFamilyDelete(cell)) { + deleteTracker.addFamilyDelete(cell); + if (clearDeletes) { + return ReturnCode.NEXT_COL; + } else { + // cell is visible + // as soon as we find a KV to include we can move to the next column + return runSubFilter(ReturnCode.INCLUDE_AND_NEXT_COL, cell); + } + } + // check if masked by family delete + if (deleteTracker.isDeleted(cell)) { + return ReturnCode.NEXT_COL; + } + // check for column delete + if (isColumnDelete(cell)) { + if (clearDeletes) { + // skip "deleted" cell + return ReturnCode.NEXT_COL; + } else { + // keep the marker but skip any remaining versions + return runSubFilter(ReturnCode.INCLUDE_AND_NEXT_COL, cell); + } + } + // cell is visible + // as soon as we find a KV to include we can move to the next column + + return runSubFilter(ReturnCode.INCLUDE_AND_NEXT_COL, cell); + } else { + return ReturnCode.SKIP; + } + } + + private ReturnCode runSubFilter(ReturnCode txFilterCode, Cell cell) throws IOException { + if (cellFilter != null) { + ReturnCode subFilterCode = cellFilter.filterKeyValue(cell); + return determineReturnCode(txFilterCode, subFilterCode); + } + return txFilterCode; + } + + /** + * Determines the return code of TransactionVisibilityFilter based on sub-filter's return code. + * Sub-filter can only exclude cells included by TransactionVisibilityFilter, i.e., sub-filter's + * INCLUDE will be ignored. This behavior makes sure that sub-filter only sees cell versions valid for the + * given transaction. If sub-filter needs to see older versions of cell, then this method can be overridden. + * + * @param txFilterCode return code from TransactionVisibilityFilter + * @param subFilterCode return code from sub-filter + * @return final return code + */ + protected ReturnCode determineReturnCode(ReturnCode txFilterCode, ReturnCode subFilterCode) { + // Return the more restrictive of the two filter responses + switch (subFilterCode) { + case INCLUDE: + return txFilterCode; + case INCLUDE_AND_NEXT_COL: + return ReturnCode.INCLUDE_AND_NEXT_COL; + case SKIP: + return txFilterCode == ReturnCode.INCLUDE ? ReturnCode.SKIP : ReturnCode.NEXT_COL; + default: + return subFilterCode; + } + } + + @Override + public boolean filterRow() throws IOException { + if (cellFilter != null) { + return cellFilter.filterRow(); + } + return super.filterRow(); + } + + @Override + public Cell transformCell(Cell cell) throws IOException { + // Convert Tephra deletes back into HBase deletes + if (tx.getVisibilityLevel() == Transaction.VisibilityLevel.SNAPSHOT_ALL) { + if (DeleteTracker.isFamilyDelete(cell)) { + return new KeyValue(CellUtil.cloneRow(cell), CellUtil.cloneFamily(cell), null, cell.getTimestamp(), + KeyValue.Type.DeleteFamily); + } else if (isColumnDelete(cell)) { + // Note: in some cases KeyValue.Type.Delete is used in Delete object, + // and in some other cases KeyValue.Type.DeleteColumn is used. + // Since Tephra cannot distinguish between the two, we return KeyValue.Type.DeleteColumn. + // KeyValue.Type.DeleteColumn makes both CellUtil.isDelete and CellUtil.isDeleteColumns return true, and will + // work in both cases. + return new KeyValue(CellUtil.cloneRow(cell), CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), + cell.getTimestamp(), KeyValue.Type.DeleteColumn); + } + } + return cell; + } + + @Override + public void reset() throws IOException { + deleteTracker.reset(); + if (cellFilter != null) { + cellFilter.reset(); + } + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { + if (cellFilter != null) { + return cellFilter.filterRowKey(buffer, offset, length); + } + return super.filterRowKey(buffer, offset, length); + } + + @Override + public boolean filterAllRemaining() throws IOException { + if (cellFilter != null) { + return cellFilter.filterAllRemaining(); + } + return super.filterAllRemaining(); + } + + @Override + public void filterRowCells(List kvs) throws IOException { + if (cellFilter != null) { + cellFilter.filterRowCells(kvs); + } else { + super.filterRowCells(kvs); + } + } + + @Override + public boolean hasFilterRow() { + if (cellFilter != null) { + return cellFilter.hasFilterRow(); + } + return super.hasFilterRow(); + } + + @Override + public Cell getNextCellHint(Cell currentKV) throws IOException { + if (cellFilter != null) { + return cellFilter.getNextCellHint(currentKV); + } + return super.getNextCellHint(currentKV); + } + + @Override + public boolean isFamilyEssential(byte[] name) throws IOException { + if (cellFilter != null) { + return cellFilter.isFamilyEssential(name); + } + return super.isFamilyEssential(name); + } + + private boolean isColumnDelete(Cell cell) { + return !TxUtils.isPreExistingVersion(cell.getTimestamp()) && cell.getValueLength() == 0 && !allowEmptyValues; + } + + private static final class DeleteTracker { + private long familyDeleteTs; + private byte[] rowKey; + + public static boolean isFamilyDelete(Cell cell) { + return !TxUtils.isPreExistingVersion(cell.getTimestamp()) && + CellUtil.matchingQualifier(cell, TxConstants.FAMILY_DELETE_QUALIFIER) && + CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY); + } + + public void addFamilyDelete(Cell delete) { + this.familyDeleteTs = delete.getTimestamp(); + this.rowKey = Bytes.copy(delete.getRowArray(), delete.getRowOffset(), delete.getRowLength()); + } + + public boolean isDeleted(Cell cell) { + return rowKey != null && Bytes.compareTo(cell.getRowArray(), cell.getRowOffset(), + cell.getRowLength(), rowKey, 0, rowKey.length) == 0 && cell.getTimestamp() <= familyDeleteTs; + } + + public void reset() { + this.familyDeleteTs = 0; + this.rowKey = null; + } + } +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java new file mode 100644 index 00000000..18d93942 --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/CompactionState.java @@ -0,0 +1,112 @@ +/* + * 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.tephra.hbase.txprune; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; +import org.apache.tephra.Transaction; +import org.apache.tephra.persist.TransactionVisibilityState; +import org.apache.tephra.util.TxUtils; + +import java.io.IOException; +import javax.annotation.Nullable; + +/** + * Record compaction state for invalid list pruning + */ +public class CompactionState { + private static final Log LOG = LogFactory.getLog(CompactionState.class); + + private final byte[] regionName; + private final String regionNameAsString; + private final PruneUpperBoundWriterSupplier pruneUpperBoundWriterSupplier; + private final PruneUpperBoundWriter pruneUpperBoundWriter; + + private volatile long pruneUpperBound = -1; + + public CompactionState(final RegionCoprocessorEnvironment env, final TableName stateTable, long pruneFlushInterval) { + this.regionName = env.getRegionInfo().getRegionName(); + this.regionNameAsString = env.getRegionInfo().getRegionNameAsString(); + DataJanitorState dataJanitorState = new DataJanitorState(new DataJanitorState.TableSupplier() { + @Override + public Table get() throws IOException { + return env.getConnection().getTable(stateTable); + } + }); + this.pruneUpperBoundWriterSupplier = new PruneUpperBoundWriterSupplier(stateTable, dataJanitorState, + pruneFlushInterval); + this.pruneUpperBoundWriter = pruneUpperBoundWriterSupplier.get(); + } + + /** + * Records the transaction state used for a compaction. This method is called when the compaction starts. + * + * @param request {@link CompactionRequest} for the compaction + * @param snapshot transaction state that will be used for the compaction + */ + public void record(CompactionRequest request, @Nullable TransactionVisibilityState snapshot) { + if (request.isMajor() && snapshot != null) { + Transaction tx = TxUtils.createDummyTransaction(snapshot); + pruneUpperBound = TxUtils.getPruneUpperBound(tx); + if (LOG.isDebugEnabled()) { + LOG.debug( + String.format("Computed prune upper bound %s for compaction request %s using transaction state from time %s", + pruneUpperBound, request, snapshot.getTimestamp())); + } + } else { + pruneUpperBound = -1; + } + } + + /** + * Persists the transaction state recorded by {@link #record(CompactionRequest, TransactionVisibilityState)}. + * This method is called after the compaction has successfully completed. + */ + public void persist() { + if (pruneUpperBound != -1) { + pruneUpperBoundWriter.persistPruneEntry(regionName, pruneUpperBound); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Enqueued prune upper bound %s for region %s", pruneUpperBound, regionNameAsString)); + } + } + } + + /** + * Persist that the given region is empty at the given time + * @param time time in milliseconds + */ + public void persistRegionEmpty(long time) { + pruneUpperBoundWriter.persistRegionEmpty(regionName, time); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Enqueued empty region %s at time %s", regionNameAsString, time)); + } + } + + /** + * Releases the usage {@link PruneUpperBoundWriter}. + */ + public void stop() { + pruneUpperBoundWriterSupplier.release(); + } +} diff --git a/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java new file mode 100644 index 00000000..db59d7d6 --- /dev/null +++ b/tephra-hbase-compat-2.0/src/main/java/org/apache/tephra/hbase/txprune/DataJanitorState.java @@ -0,0 +1,536 @@ +/* + * 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.tephra.hbase.txprune; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.tephra.hbase.coprocessor.TransactionProcessor; +import org.apache.tephra.txprune.RegionPruneInfo; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import javax.annotation.Nullable; + +/** + * Persist data janitor state into an HBase table. + * This is used by both {@link TransactionProcessor} and by the {@link HBaseTransactionPruningPlugin} + * to persist and read the compaction state. + */ +@SuppressWarnings("WeakerAccess") +public class DataJanitorState { + private static final Log LOG = LogFactory.getLog(DataJanitorState.class); + + public static final byte[] FAMILY = {'f'}; + public static final byte[] PRUNE_UPPER_BOUND_COL = {'p'}; + + private static final byte[] REGION_TIME_COL = {'r'}; + private static final byte[] INACTIVE_TRANSACTION_BOUND_TIME_COL = {'i'}; + private static final byte[] EMPTY_REGION_TIME_COL = {'e'}; + + private static final byte[] REGION_KEY_PREFIX = {0x1}; + private static final byte[] REGION_KEY_PREFIX_STOP = {0x2}; + + private static final byte[] REGION_TIME_KEY_PREFIX = {0x2}; + private static final byte[] REGION_TIME_KEY_PREFIX_STOP = {0x3}; + + private static final byte[] INACTIVE_TRANSACTION_BOUND_TIME_KEY_PREFIX = {0x3}; + private static final byte[] INACTIVE_TRANSACTION_BOUND_TIME_KEY_PREFIX_STOP = {0x4}; + + private static final byte[] EMPTY_REGION_TIME_KEY_PREFIX = {0x4}; + private static final byte[] EMPTY_REGION_TIME_KEY_PREFIX_STOP = {0x5}; + + private static final byte[] REGION_TIME_COUNT_KEY_PREFIX = {0x5}; + private static final byte[] REGION_TIME_COUNT_KEY_PREFIX_STOP = {0x6}; + + private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; + // This value can be used when we don't care about the value we write in a column + private static final byte[] COL_VAL = Bytes.toBytes('1'); + + private final TableSupplier stateTableSupplier; + + + public DataJanitorState(TableSupplier stateTableSupplier) { + this.stateTableSupplier = stateTableSupplier; + } + + // ---------------------------------------------------------------- + // ------- Methods for prune upper bound for a given region ------- + // ---------------------------------------------------------------- + // The data is stored in the following format - + // Key: 0x1 + // Col 'u': + // ---------------------------------------------------------------- + + /** + * Persist the latest prune upper bound for a given region. This is called by {@link TransactionProcessor} + * after major compaction. + * + * @param regionId region id + * @param pruneUpperBound the latest prune upper bound for the region + * @throws IOException when not able to persist the data to HBase + */ + public void savePruneUpperBoundForRegion(byte[] regionId, long pruneUpperBound) throws IOException { + try (Table stateTable = stateTableSupplier.get()) { + Put put = new Put(makeRegionKey(regionId)); + put.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL, Bytes.toBytes(pruneUpperBound)); + stateTable.put(put); + } + } + + /** + * Get latest prune upper bound for a given region. This indicates the largest invalid transaction that no + * longer has writes in this region. + * + * @param regionId region id + * @return latest prune upper bound for the region + * @throws IOException when not able to read the data from HBase + */ + public long getPruneUpperBoundForRegion(byte[] regionId) throws IOException { + RegionPruneInfo regionPruneInfo = getPruneInfoForRegion(regionId); + return (regionPruneInfo == null) ? -1 : regionPruneInfo.getPruneUpperBound(); + } + + /** + * Get the latest {@link RegionPruneInfo} for a given region. + * + * @param regionId region id + * @return {@link RegionPruneInfo} for the region + * @throws IOException when not able to read the data from HBase + */ + @Nullable + public RegionPruneInfo getPruneInfoForRegion(byte[] regionId) throws IOException { + try (Table stateTable = stateTableSupplier.get()) { + Get get = new Get(makeRegionKey(regionId)); + get.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL); + Cell cell = stateTable.get(get).getColumnLatestCell(FAMILY, PRUNE_UPPER_BOUND_COL); + if (cell == null) { + return null; + } + byte[] pruneUpperBoundBytes = CellUtil.cloneValue(cell); + long timestamp = cell.getTimestamp(); + return new RegionPruneInfo(regionId, Bytes.toStringBinary(regionId), + Bytes.toLong(pruneUpperBoundBytes), timestamp); + } + } + + /** + * Get latest prune upper bounds for given regions. This is a batch operation of method + * {@link #getPruneUpperBoundForRegion(byte[])} + * + * @param regions a set of regions + * @return a map containing region id and its latest prune upper bound value + * @throws IOException when not able to read the data from HBase + */ + public Map getPruneUpperBoundForRegions(SortedSet regions) throws IOException { + Map resultMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); + List regionPruneInfos = getPruneInfoForRegions(regions); + for (RegionPruneInfo regionPruneInfo : regionPruneInfos) { + resultMap.put(regionPruneInfo.getRegionName(), regionPruneInfo.getPruneUpperBound()); + } + return Collections.unmodifiableMap(resultMap); + } + + /** + * Gets a list of {@link RegionPruneInfo} for given regions. Returns all regions if the given regions set is null. + * + * @param regions a set of regions + * @return list of {@link RegionPruneInfo}s. + * @throws IOException when not able to read the data from HBase + */ + public List getPruneInfoForRegions(@Nullable SortedSet regions) throws IOException { + List regionPruneInfos = new ArrayList<>(); + try (Table stateTable = stateTableSupplier.get()) { + byte[] startRow = makeRegionKey(EMPTY_BYTE_ARRAY); + Scan scan = new Scan(startRow, REGION_KEY_PREFIX_STOP); + scan.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL); + + try (ResultScanner scanner = stateTable.getScanner(scan)) { + Result next; + while ((next = scanner.next()) != null) { + byte[] region = getRegionFromKey(next.getRow()); + if (regions == null || regions.contains(region)) { + Cell cell = next.getColumnLatestCell(FAMILY, PRUNE_UPPER_BOUND_COL); + if (cell != null) { + byte[] pruneUpperBoundBytes = CellUtil.cloneValue(cell); + long timestamp = cell.getTimestamp(); + regionPruneInfos.add(new RegionPruneInfo(region, Bytes.toStringBinary(region), + Bytes.toLong(pruneUpperBoundBytes), timestamp)); + } + } + } + } + } + return Collections.unmodifiableList(regionPruneInfos); + } + + /** + * Delete prune upper bounds for the regions that are not in the given exclude set, and the + * prune upper bound is less than the given value. + * After the invalid list is pruned up to deletionPruneUpperBound, we do not need entries for regions that have + * prune upper bound less than deletionPruneUpperBound. We however limit the deletion to only regions that are + * no longer in existence (due to deletion, etc.), to avoid update/delete race conditions. + * + * @param deletionPruneUpperBound prune upper bound below which regions will be deleted + * @param excludeRegions set of regions that should not be deleted + * @throws IOException when not able to delete data in HBase + */ + public void deletePruneUpperBounds(long deletionPruneUpperBound, SortedSet excludeRegions) + throws IOException { + try (Table stateTable = stateTableSupplier.get()) { + byte[] startRow = makeRegionKey(EMPTY_BYTE_ARRAY); + Scan scan = new Scan(startRow, REGION_KEY_PREFIX_STOP); + scan.addColumn(FAMILY, PRUNE_UPPER_BOUND_COL); + + try (ResultScanner scanner = stateTable.getScanner(scan)) { + Result next; + while ((next = scanner.next()) != null) { + byte[] region = getRegionFromKey(next.getRow()); + if (!excludeRegions.contains(region)) { + byte[] timeBytes = next.getValue(FAMILY, PRUNE_UPPER_BOUND_COL); + if (timeBytes != null) { + long pruneUpperBoundRegion = Bytes.toLong(timeBytes); + if (pruneUpperBoundRegion < deletionPruneUpperBound) { + stateTable.delete(new Delete(next.getRow())); + } + } + } + } + } + } + } + + // --------------------------------------------------- + // ------- Methods for regions at a given time ------- + // --------------------------------------------------- + // Key: 0x2 + // Col 't': + // --------------------------------------------------- + + /** + * Persist the regions for the given time. {@link HBaseTransactionPruningPlugin} saves the set of + * transactional regions existing in the HBase instance periodically. + * + * @param time timestamp in milliseconds + * @param regions set of regions at the time + * @throws IOException when not able to persist the data to HBase + */ + public void saveRegionsForTime(long time, Set regions) throws IOException { + byte[] timeBytes = Bytes.toBytes(getInvertedTime(time)); + try (Table stateTable = stateTableSupplier.get()) { + for (byte[] region : regions) { + Put put = new Put(makeTimeRegionKey(timeBytes, region)); + put.addColumn(FAMILY, REGION_TIME_COL, COL_VAL); + stateTable.put(put); + } + + // Save the count of regions as a checksum + saveRegionCountForTime(stateTable, timeBytes, regions.size()); + } + } + + @VisibleForTesting + void saveRegionCountForTime(Table stateTable, byte[] timeBytes, int count) throws IOException { + Put put = new Put(makeTimeRegionCountKey(timeBytes)); + put.addColumn(FAMILY, REGION_TIME_COL, Bytes.toBytes(count)); + stateTable.put(put); + } + + /** + * Return the set of regions saved for the time at or before the given time. This method finds the greatest time + * that is less than or equal to the given time, and then returns all regions with that exact time, but none that are + * older than that. + * + * @param time timestamp in milliseconds + * @return set of regions and time at which they were recorded, or null if no regions found + * @throws IOException when not able to read the data from HBase + */ + @Nullable + public TimeRegions getRegionsOnOrBeforeTime(long time) throws IOException { + try (Table stateTable = stateTableSupplier.get()) { + TimeRegions timeRegions; + while ((timeRegions = getNextSetOfTimeRegions(stateTable, time)) != null) { + int count = getRegionCountForTime(stateTable, timeRegions.getTime()); + if (count != -1 && count == timeRegions.getRegions().size()) { + return timeRegions; + } else { + LOG.warn(String.format("Got incorrect count for regions saved at time %s, expected = %s but actual = %s", + timeRegions.getTime(), count, timeRegions.getRegions().size())); + time = timeRegions.getTime() - 1; + } + } + return null; + } + } + + @Nullable + private TimeRegions getNextSetOfTimeRegions(Table stateTable, long time) throws IOException { + byte[] timeBytes = Bytes.toBytes(getInvertedTime(time)); + Scan scan = new Scan(makeTimeRegionKey(timeBytes, EMPTY_BYTE_ARRAY), REGION_TIME_KEY_PREFIX_STOP); + scan.addColumn(FAMILY, REGION_TIME_COL); + + + long currentRegionTime = -1; + SortedSet regions = new TreeSet<>(Bytes.BYTES_COMPARATOR); + Result next; + try (ResultScanner scanner = stateTable.getScanner(scan)) { + while ((next = scanner.next()) != null) { + Map.Entry timeRegion = getTimeRegion(next.getRow()); + // Stop if reached next time value + if (currentRegionTime == -1) { + currentRegionTime = timeRegion.getKey(); + } else if (timeRegion.getKey() < currentRegionTime) { + break; + } else if (timeRegion.getKey() > currentRegionTime) { + throw new IllegalStateException( + String.format("Got out of order time %d when expecting time less than or equal to %d", + timeRegion.getKey(), currentRegionTime)); + } + regions.add(timeRegion.getValue()); + } + } + return regions.isEmpty() ? null : new TimeRegions(currentRegionTime, Collections.unmodifiableSortedSet(regions)); + } + + @VisibleForTesting + int getRegionCountForTime(Table stateTable, long time) throws IOException { + Get get = new Get(makeTimeRegionCountKey(Bytes.toBytes(getInvertedTime(time)))); + get.addColumn(FAMILY, REGION_TIME_COL); + Result result = stateTable.get(get); + byte[] value = result.getValue(FAMILY, REGION_TIME_COL); + return value == null ? -1 : Bytes.toInt(value); + } + + /** + * Delete all the regions that were recorded for all times equal or less than the given time. + * + * @param time timestamp in milliseconds + * @throws IOException when not able to delete data in HBase + */ + public void deleteAllRegionsOnOrBeforeTime(long time) throws IOException { + byte[] timeBytes = Bytes.toBytes(getInvertedTime(time)); + try (Table stateTable = stateTableSupplier.get()) { + // Delete the regions + Scan scan = new Scan(makeTimeRegionKey(timeBytes, EMPTY_BYTE_ARRAY), REGION_TIME_KEY_PREFIX_STOP); + scan.addColumn(FAMILY, REGION_TIME_COL); + deleteFromScan(stateTable, scan); + + // Delete the count + scan = new Scan(makeTimeRegionCountKey(timeBytes), REGION_TIME_COUNT_KEY_PREFIX_STOP); + scan.addColumn(FAMILY, REGION_TIME_COL); + deleteFromScan(stateTable, scan); + } + } + + // --------------------------------------------------------------------- + // ------- Methods for inactive transaction bound for given time ------- + // --------------------------------------------------------------------- + // Key: 0x3 + // Col 'p': + // --------------------------------------------------------------------- + + /** + * Persist inactive transaction bound for a given time. This is the smallest not in-progress transaction that + * will not have writes in any HBase regions that are created after the given time. + * + * @param time time in milliseconds + * @param inactiveTransactionBound inactive transaction bound for the given time + * @throws IOException when not able to persist the data to HBase + */ + public void saveInactiveTransactionBoundForTime(long time, long inactiveTransactionBound) throws IOException { + try (Table stateTable = stateTableSupplier.get()) { + Put put = new Put(makeInactiveTransactionBoundTimeKey(Bytes.toBytes(getInvertedTime(time)))); + put.addColumn(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL, Bytes.toBytes(inactiveTransactionBound)); + stateTable.put(put); + } + } + + /** + * Return inactive transaction bound for the given time. + * + * @param time time in milliseconds + * @return inactive transaction bound for the given time + * @throws IOException when not able to read the data from HBase + */ + public long getInactiveTransactionBoundForTime(long time) throws IOException { + try (Table stateTable = stateTableSupplier.get()) { + Get get = new Get(makeInactiveTransactionBoundTimeKey(Bytes.toBytes(getInvertedTime(time)))); + get.addColumn(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL); + byte[] result = stateTable.get(get).getValue(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL); + return result == null ? -1 : Bytes.toLong(result); + } + } + + /** + * Delete all inactive transaction bounds recorded for a time less than the given time + * + * @param time time in milliseconds + * @throws IOException when not able to delete data in HBase + */ + public void deleteInactiveTransactionBoundsOnOrBeforeTime(long time) throws IOException { + try (Table stateTable = stateTableSupplier.get()) { + Scan scan = new Scan(makeInactiveTransactionBoundTimeKey(Bytes.toBytes(getInvertedTime(time))), + INACTIVE_TRANSACTION_BOUND_TIME_KEY_PREFIX_STOP); + scan.addColumn(FAMILY, INACTIVE_TRANSACTION_BOUND_TIME_COL); + deleteFromScan(stateTable, scan); + } + } + + // -------------------------------------------------------- + // ------- Methods for empty regions at a given time ------- + // -------------------------------------------------------- + // Key: 0x4