You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/07/24 21:49:20 UTC
[1/6] ACCUMULO-1000 added conditional mutations to Accumulo
Updated Branches:
refs/heads/master 77cac56ec -> 9dc244484
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
new file mode 100644
index 0000000..a42a08b
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
@@ -0,0 +1,1236 @@
+/*
+ * 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.accumulo.test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriter.Result;
+import org.apache.accumulo.core.client.ConditionalWriter.Status;
+import org.apache.accumulo.core.client.ConditionalWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IsolatedScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Condition;
+import org.apache.accumulo.core.data.ConditionalMutation;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.LongCombiner.Type;
+import org.apache.accumulo.core.iterators.user.SummingCombiner;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.CredentialHelper;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.FastFormat;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.examples.simple.constraints.AlphaNumKeyConstraint;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.test.functional.BadIterator;
+import org.apache.accumulo.test.functional.SlowIterator;
+import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+
+/**
+ *
+ */
+public class ConditionalWriterTest {
+
+ private static String secret = "superSecret";
+ public static TemporaryFolder folder = new TemporaryFolder();
+ public static MiniAccumuloCluster cluster;
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ folder.create();
+ MiniAccumuloConfig cfg = new MiniAccumuloConfig(folder.newFolder("miniAccumulo"), secret);
+ cluster = new MiniAccumuloCluster(cfg);
+ cluster.start();
+ }
+
+ @Test
+ public void testBasic() throws Exception {
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create("foo");
+
+ ConditionalWriter cw = conn.createConditionalWriter("foo", new ConditionalWriterConfig());
+
+ // mutation conditional on column tx:seq not exiting
+ ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq"));
+ cm0.put("name", "last", "doe");
+ cm0.put("name", "first", "john");
+ cm0.put("tx", "seq", "1");
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm0).getStatus());
+ Assert.assertEquals(Status.REJECTED, cw.write(cm0).getStatus());
+
+ // mutation conditional on column tx:seq being 1
+ ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("1"));
+ cm1.put("name", "last", "Doe");
+ cm1.put("tx", "seq", "2");
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+
+ // test condition where value differs
+ ConditionalMutation cm2 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("1"));
+ cm2.put("name", "last", "DOE");
+ cm2.put("tx", "seq", "2");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm2).getStatus());
+
+ // test condition where column does not exists
+ ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("txtypo", "seq").setValue("1"));
+ cm3.put("name", "last", "deo");
+ cm3.put("tx", "seq", "2");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm3).getStatus());
+
+ // test two conditions, where one should fail
+ ConditionalMutation cm4 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("2"), new Condition("name", "last").setValue("doe"));
+ cm4.put("name", "last", "deo");
+ cm4.put("tx", "seq", "3");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm4).getStatus());
+
+ // test two conditions, where one should fail
+ ConditionalMutation cm5 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("1"), new Condition("name", "last").setValue("Doe"));
+ cm5.put("name", "last", "deo");
+ cm5.put("tx", "seq", "3");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm5).getStatus());
+
+ // ensure rejected mutations did not write
+ Scanner scanner = conn.createScanner("foo", Authorizations.EMPTY);
+ scanner.fetchColumn(new Text("name"), new Text("last"));
+ scanner.setRange(new Range("99006"));
+ Assert.assertEquals("Doe", scanner.iterator().next().getValue().toString());
+
+ // test w/ two conditions that are met
+ ConditionalMutation cm6 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("2"), new Condition("name", "last").setValue("Doe"));
+ cm6.put("name", "last", "DOE");
+ cm6.put("tx", "seq", "3");
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm6).getStatus());
+
+ Assert.assertEquals("DOE", scanner.iterator().next().getValue().toString());
+
+ // test a conditional mutation that deletes
+ ConditionalMutation cm7 = new ConditionalMutation("99006", new Condition("tx", "seq").setValue("3"));
+ cm7.putDelete("name", "last");
+ cm7.putDelete("name", "first");
+ cm7.putDelete("tx", "seq");
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm7).getStatus());
+
+ Assert.assertFalse(scanner.iterator().hasNext());
+
+ // add the row back
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm0).getStatus());
+ Assert.assertEquals(Status.REJECTED, cw.write(cm0).getStatus());
+
+ Assert.assertEquals("doe", scanner.iterator().next().getValue().toString());
+ }
+
+ @Test
+ public void testFields() throws Exception {
+ String table = "foo2";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+
+ Authorizations auths = new Authorizations("A", "B");
+
+ conn.securityOperations().changeUserAuthorizations("root", auths);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(auths));
+
+ ColumnVisibility cva = new ColumnVisibility("A");
+ ColumnVisibility cvb = new ColumnVisibility("B");
+
+ ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva));
+ cm0.put("name", "last", cva, "doe");
+ cm0.put("name", "first", cva, "john");
+ cm0.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm0).getStatus());
+
+ Scanner scanner = conn.createScanner(table, auths);
+ scanner.setRange(new Range("99006"));
+ // TODO verify all columns
+ scanner.fetchColumn(new Text("tx"), new Text("seq"));
+ Entry<Key,Value> entry = scanner.iterator().next();
+ Assert.assertEquals("1", entry.getValue().toString());
+ long ts = entry.getKey().getTimestamp();
+
+ // test wrong colf
+ ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("txA", "seq").setVisibility(cva).setValue("1"));
+ cm1.put("name", "last", cva, "Doe");
+ cm1.put("name", "first", cva, "John");
+ cm1.put("tx", "seq", cva, "2");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm1).getStatus());
+
+ // test wrong colq
+ ConditionalMutation cm2 = new ConditionalMutation("99006", new Condition("tx", "seqA").setVisibility(cva).setValue("1"));
+ cm2.put("name", "last", cva, "Doe");
+ cm2.put("name", "first", cva, "John");
+ cm2.put("tx", "seq", cva, "2");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm2).getStatus());
+
+ // test wrong colv
+ ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"));
+ cm3.put("name", "last", cva, "Doe");
+ cm3.put("name", "first", cva, "John");
+ cm3.put("tx", "seq", cva, "2");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm3).getStatus());
+
+ // test wrong timestamp
+ ConditionalMutation cm4 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva).setTimestamp(ts + 1).setValue("1"));
+ cm4.put("name", "last", cva, "Doe");
+ cm4.put("name", "first", cva, "John");
+ cm4.put("tx", "seq", cva, "2");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm4).getStatus());
+
+ // test wrong timestamp
+ ConditionalMutation cm5 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva).setTimestamp(ts - 1).setValue("1"));
+ cm5.put("name", "last", cva, "Doe");
+ cm5.put("name", "first", cva, "John");
+ cm5.put("tx", "seq", cva, "2");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm5).getStatus());
+
+ // ensure no updates were made
+ entry = scanner.iterator().next();
+ Assert.assertEquals("1", entry.getValue().toString());
+
+ // set all columns correctly
+ ConditionalMutation cm6 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cva).setTimestamp(ts).setValue("1"));
+ cm6.put("name", "last", cva, "Doe");
+ cm6.put("name", "first", cva, "John");
+ cm6.put("tx", "seq", cva, "2");
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm6).getStatus());
+
+ entry = scanner.iterator().next();
+ Assert.assertEquals("2", entry.getValue().toString());
+
+ // TODO test each field w/ absence
+
+ }
+
+ @Test
+ public void testBadColVis() throws Exception {
+ // test when a user sets a col vis in a condition that can never be seen
+ String table = "foo3";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+
+ Authorizations auths = new Authorizations("A", "B");
+
+ conn.securityOperations().changeUserAuthorizations("root", auths);
+
+ Authorizations filteredAuths = new Authorizations("A");
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(filteredAuths));
+
+ ColumnVisibility cva = new ColumnVisibility("A");
+ ColumnVisibility cvb = new ColumnVisibility("B");
+ ColumnVisibility cvc = new ColumnVisibility("C");
+
+ // User has authorization, but didn't include it in the writer
+ ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb));
+ cm0.put("name", "last", cva, "doe");
+ cm0.put("name", "first", cva, "john");
+ cm0.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm0).getStatus());
+
+ ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"));
+ cm1.put("name", "last", cva, "doe");
+ cm1.put("name", "first", cva, "john");
+ cm1.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm1).getStatus());
+
+ // User does not have the authorization
+ ConditionalMutation cm2 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvc));
+ cm2.put("name", "last", cva, "doe");
+ cm2.put("name", "first", cva, "john");
+ cm2.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm2).getStatus());
+
+ ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvc).setValue("1"));
+ cm3.put("name", "last", cva, "doe");
+ cm3.put("name", "first", cva, "john");
+ cm3.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm3).getStatus());
+
+ // if any visibility is bad, good visibilities don't override
+ ConditionalMutation cm4 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb), new Condition("tx", "seq").setVisibility(cva));
+
+ cm4.put("name", "last", cva, "doe");
+ cm4.put("name", "first", cva, "john");
+ cm4.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm4).getStatus());
+
+ ConditionalMutation cm5 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"), new Condition("tx", "seq")
+ .setVisibility(cva).setValue("1"));
+ cm5.put("name", "last", cva, "doe");
+ cm5.put("name", "first", cva, "john");
+ cm5.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm5).getStatus());
+
+ ConditionalMutation cm6 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb).setValue("1"),
+ new Condition("tx", "seq").setVisibility(cva));
+ cm6.put("name", "last", cva, "doe");
+ cm6.put("name", "first", cva, "john");
+ cm6.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm6).getStatus());
+
+ ConditionalMutation cm7 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb), new Condition("tx", "seq").setVisibility(cva)
+ .setValue("1"));
+ cm7.put("name", "last", cva, "doe");
+ cm7.put("name", "first", cva, "john");
+ cm7.put("tx", "seq", cva, "1");
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, cw.write(cm7).getStatus());
+
+ cw.close();
+
+ // test passing auths that exceed users configured auths
+
+ Authorizations exceedingAuths = new Authorizations("A", "B", "D");
+ ConditionalWriter cw2 = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(exceedingAuths));
+
+ ConditionalMutation cm8 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvb), new Condition("tx", "seq").setVisibility(cva)
+ .setValue("1"));
+ cm8.put("name", "last", cva, "doe");
+ cm8.put("name", "first", cva, "john");
+ cm8.put("tx", "seq", cva, "1");
+
+ try {
+ cw2.write(cm8).getStatus();
+ Assert.assertTrue(false);
+ } catch (AccumuloSecurityException ase) {}
+
+
+ cw2.close();
+ }
+
+ @Test
+ public void testConstraints() throws Exception {
+ // ensure constraint violations are properly reported
+ String table = "foo5";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+ conn.tableOperations().addConstraint(table, AlphaNumKeyConstraint.class.getName());
+ conn.tableOperations().clone(table, table + "_clone", true, new HashMap<String,String>(), new HashSet<String>());
+
+ Scanner scanner = conn.createScanner(table + "_clone", new Authorizations());
+
+ ConditionalWriter cw = conn.createConditionalWriter(table + "_clone", new ConditionalWriterConfig());
+
+ ConditionalMutation cm0 = new ConditionalMutation("99006+", new Condition("tx", "seq"));
+ cm0.put("tx", "seq", "1");
+
+ Assert.assertEquals(Status.VIOLATED, cw.write(cm0).getStatus());
+ Assert.assertFalse(scanner.iterator().hasNext());
+
+ ConditionalMutation cm1 = new ConditionalMutation("99006", new Condition("tx", "seq"));
+ cm1.put("tx", "seq", "1");
+
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+ Assert.assertTrue(scanner.iterator().hasNext());
+
+ cw.close();
+
+ }
+
+ @Test
+ public void testIterators() throws Exception {
+ String table = "foo4";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table, false);
+
+ BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+
+ Mutation m = new Mutation("ACCUMULO-1000");
+ m.put("count", "comments", "1");
+ bw.addMutation(m);
+ bw.addMutation(m);
+ bw.addMutation(m);
+
+ m = new Mutation("ACCUMULO-1001");
+ m.put("count2", "comments", "1");
+ bw.addMutation(m);
+ bw.addMutation(m);
+
+ m = new Mutation("ACCUMULO-1002");
+ m.put("count2", "comments", "1");
+ bw.addMutation(m);
+ bw.addMutation(m);
+
+ bw.close();
+
+ IteratorSetting iterConfig = new IteratorSetting(10, SummingCombiner.class);
+ SummingCombiner.setEncodingType(iterConfig, Type.STRING);
+ SummingCombiner.setColumns(iterConfig, Collections.singletonList(new IteratorSetting.Column("count")));
+
+ IteratorSetting iterConfig2 = new IteratorSetting(10, SummingCombiner.class);
+ SummingCombiner.setEncodingType(iterConfig2, Type.STRING);
+ SummingCombiner.setColumns(iterConfig2, Collections.singletonList(new IteratorSetting.Column("count2", "comments")));
+
+ IteratorSetting iterConfig3 = new IteratorSetting(5, VersioningIterator.class);
+ VersioningIterator.setMaxVersions(iterConfig3, 1);
+
+ Scanner scanner = conn.createScanner(table, new Authorizations());
+ scanner.addScanIterator(iterConfig);
+ scanner.setRange(new Range("ACCUMULO-1000"));
+ scanner.fetchColumn(new Text("count"), new Text("comments"));
+
+ Assert.assertEquals("3", scanner.iterator().next().getValue().toString());
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+ ConditionalMutation cm0 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setValue("3"));
+ cm0.put("count", "comments", "1");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm0).getStatus());
+ Assert.assertEquals("3", scanner.iterator().next().getValue().toString());
+
+ ConditionalMutation cm1 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setIterators(iterConfig).setValue("3"));
+ cm1.put("count", "comments", "1");
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+ Assert.assertEquals("4", scanner.iterator().next().getValue().toString());
+
+ ConditionalMutation cm2 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setValue("4"));
+ cm2.put("count", "comments", "1");
+ Assert.assertEquals(Status.REJECTED, cw.write(cm1).getStatus());
+ Assert.assertEquals("4", scanner.iterator().next().getValue().toString());
+
+ // run test with multiple iterators passed in same batch and condition with two iterators
+
+ ConditionalMutation cm3 = new ConditionalMutation("ACCUMULO-1000", new Condition("count", "comments").setIterators(iterConfig).setValue("4"));
+ cm3.put("count", "comments", "1");
+
+ ConditionalMutation cm4 = new ConditionalMutation("ACCUMULO-1001", new Condition("count2", "comments").setIterators(iterConfig2).setValue("2"));
+ cm4.put("count2", "comments", "1");
+
+ ConditionalMutation cm5 = new ConditionalMutation("ACCUMULO-1002", new Condition("count2", "comments").setIterators(iterConfig2, iterConfig3).setValue("2"));
+ cm5.put("count2", "comments", "1");
+
+ Iterator<Result> results = cw.write(Arrays.asList(cm3, cm4, cm5).iterator());
+ Map<String,Status> actual = new HashMap<String,Status>();
+
+ while (results.hasNext()) {
+ Result result = results.next();
+ String k = new String(result.getMutation().getRow());
+ Assert.assertFalse(actual.containsKey(k));
+ actual.put(k, result.getStatus());
+ }
+
+ Map<String,Status> expected = new HashMap<String,Status>();
+ expected.put("ACCUMULO-1000", Status.ACCEPTED);
+ expected.put("ACCUMULO-1001", Status.ACCEPTED);
+ expected.put("ACCUMULO-1002", Status.REJECTED);
+
+ Assert.assertEquals(expected, actual);
+
+ // TODO test w/ table that has iterators configured
+
+ cw.close();
+ }
+
+ @Test
+ public void testBatch() throws Exception {
+ String table = "foo6";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+
+ conn.securityOperations().changeUserAuthorizations("root", new Authorizations("A", "B"));
+
+ ColumnVisibility cvab = new ColumnVisibility("A|B");
+
+ ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+
+ ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvab));
+ cm0.put("name", "last", cvab, "doe");
+ cm0.put("name", "first", cvab, "john");
+ cm0.put("tx", "seq", cvab, "1");
+ mutations.add(cm0);
+
+ ConditionalMutation cm1 = new ConditionalMutation("59056", new Condition("tx", "seq").setVisibility(cvab));
+ cm1.put("name", "last", cvab, "doe");
+ cm1.put("name", "first", cvab, "jane");
+ cm1.put("tx", "seq", cvab, "1");
+ mutations.add(cm1);
+
+ ConditionalMutation cm2 = new ConditionalMutation("19059", new Condition("tx", "seq").setVisibility(cvab));
+ cm2.put("name", "last", cvab, "doe");
+ cm2.put("name", "first", cvab, "jack");
+ cm2.put("tx", "seq", cvab, "1");
+ mutations.add(cm2);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(new Authorizations("A")));
+ Iterator<Result> results = cw.write(mutations.iterator());
+ int count = 0;
+ while (results.hasNext()) {
+ Result result = results.next();
+ Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+ count++;
+ }
+
+ Assert.assertEquals(3, count);
+
+ Scanner scanner = conn.createScanner(table, new Authorizations("A"));
+ scanner.fetchColumn(new Text("tx"), new Text("seq"));
+
+ for (String row : new String[] {"99006", "59056", "19059"}) {
+ scanner.setRange(new Range(row));
+ Assert.assertEquals("1", scanner.iterator().next().getValue().toString());
+ }
+
+ TreeSet<Text> splits = new TreeSet<Text>();
+ splits.add(new Text("7"));
+ splits.add(new Text("3"));
+ conn.tableOperations().addSplits(table, splits);
+
+ mutations.clear();
+
+ ConditionalMutation cm3 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvab).setValue("1"));
+ cm3.put("name", "last", cvab, "Doe");
+ cm3.put("tx", "seq", cvab, "2");
+ mutations.add(cm3);
+
+ ConditionalMutation cm4 = new ConditionalMutation("59056", new Condition("tx", "seq").setVisibility(cvab));
+ cm4.put("name", "last", cvab, "Doe");
+ cm4.put("tx", "seq", cvab, "1");
+ mutations.add(cm4);
+
+ ConditionalMutation cm5 = new ConditionalMutation("19059", new Condition("tx", "seq").setVisibility(cvab).setValue("2"));
+ cm5.put("name", "last", cvab, "Doe");
+ cm5.put("tx", "seq", cvab, "3");
+ mutations.add(cm5);
+
+ results = cw.write(mutations.iterator());
+ int accepted = 0;
+ int rejected = 0;
+ while (results.hasNext()) {
+ Result result = results.next();
+ if (new String(result.getMutation().getRow()).equals("99006")) {
+ Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+ accepted++;
+ } else {
+ Assert.assertEquals(Status.REJECTED, result.getStatus());
+ rejected++;
+ }
+ }
+
+ Assert.assertEquals(1, accepted);
+ Assert.assertEquals(2, rejected);
+
+ for (String row : new String[] {"59056", "19059"}) {
+ scanner.setRange(new Range(row));
+ Assert.assertEquals("1", scanner.iterator().next().getValue().toString());
+ }
+
+ scanner.setRange(new Range("99006"));
+ Assert.assertEquals("2", scanner.iterator().next().getValue().toString());
+
+ scanner.clearColumns();
+ scanner.fetchColumn(new Text("name"), new Text("last"));
+ Assert.assertEquals("Doe", scanner.iterator().next().getValue().toString());
+
+ cw.close();
+ }
+
+ @Test
+ public void testBigBatch() throws Exception {
+
+ String table = "foo100";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+ conn.tableOperations().addSplits(table, nss("2", "4", "6"));
+
+ UtilWaitThread.sleep(2000);
+
+ int num = 100;
+
+ ArrayList<byte[]> rows = new ArrayList<byte[]>(num);
+ ArrayList<ConditionalMutation> cml = new ArrayList<ConditionalMutation>(num);
+
+ Random r = new Random();
+ byte[] e = new byte[0];
+
+ for (int i = 0; i < num; i++) {
+ rows.add(FastFormat.toZeroPaddedString(Math.abs(r.nextLong()), 16, 16, e));
+ }
+
+ for (int i = 0; i < num; i++) {
+ ConditionalMutation cm = new ConditionalMutation(rows.get(i), new Condition("meta", "seq"));
+
+ cm.put("meta", "seq", "1");
+ cm.put("meta", "tx", UUID.randomUUID().toString());
+
+ cml.add(cm);
+ }
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+ Iterator<Result> results = cw.write(cml.iterator());
+
+ int count = 0;
+
+ // TODO check got each row back
+ while (results.hasNext()) {
+ Result result = results.next();
+ Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+ count++;
+ }
+
+ Assert.assertEquals(num, count);
+
+ ArrayList<ConditionalMutation> cml2 = new ArrayList<ConditionalMutation>(num);
+
+ for (int i = 0; i < num; i++) {
+ ConditionalMutation cm = new ConditionalMutation(rows.get(i), new Condition("meta", "seq").setValue("1"));
+
+ cm.put("meta", "seq", "2");
+ cm.put("meta", "tx", UUID.randomUUID().toString());
+
+ cml2.add(cm);
+ }
+
+ count = 0;
+
+ results = cw.write(cml2.iterator());
+
+ while (results.hasNext()) {
+ Result result = results.next();
+ Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+ count++;
+ }
+
+ Assert.assertEquals(num, count);
+
+ cw.close();
+ }
+
+ @Test
+ public void testBatchErrors() throws Exception {
+
+ String table = "foo7";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+ conn.tableOperations().addConstraint(table, AlphaNumKeyConstraint.class.getName());
+ conn.tableOperations().clone(table, table + "_clone", true, new HashMap<String,String>(), new HashSet<String>());
+
+ conn.securityOperations().changeUserAuthorizations("root", new Authorizations("A", "B"));
+
+ ColumnVisibility cvaob = new ColumnVisibility("A|B");
+ ColumnVisibility cvaab = new ColumnVisibility("A&B");
+
+ switch ((new Random()).nextInt(3)) {
+ case 1:
+ conn.tableOperations().addSplits(table, nss("6"));
+ break;
+ case 2:
+ conn.tableOperations().addSplits(table, nss("2", "95"));
+ break;
+ }
+
+ ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+
+ ConditionalMutation cm0 = new ConditionalMutation("99006", new Condition("tx", "seq").setVisibility(cvaob));
+ cm0.put("name+", "last", cvaob, "doe");
+ cm0.put("name", "first", cvaob, "john");
+ cm0.put("tx", "seq", cvaob, "1");
+ mutations.add(cm0);
+
+ ConditionalMutation cm1 = new ConditionalMutation("59056", new Condition("tx", "seq").setVisibility(cvaab));
+ cm1.put("name", "last", cvaab, "doe");
+ cm1.put("name", "first", cvaab, "jane");
+ cm1.put("tx", "seq", cvaab, "1");
+ mutations.add(cm1);
+
+ ConditionalMutation cm2 = new ConditionalMutation("19059", new Condition("tx", "seq").setVisibility(cvaob));
+ cm2.put("name", "last", cvaob, "doe");
+ cm2.put("name", "first", cvaob, "jack");
+ cm2.put("tx", "seq", cvaob, "1");
+ mutations.add(cm2);
+
+ ConditionalMutation cm3 = new ConditionalMutation("90909", new Condition("tx", "seq").setVisibility(cvaob).setValue("1"));
+ cm3.put("name", "last", cvaob, "doe");
+ cm3.put("name", "first", cvaob, "john");
+ cm3.put("tx", "seq", cvaob, "2");
+ mutations.add(cm3);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setAuthorizations(new Authorizations("A")));
+ Iterator<Result> results = cw.write(mutations.iterator());
+ HashSet<String> rows = new HashSet<String>();
+ while (results.hasNext()) {
+ Result result = results.next();
+ String row = new String(result.getMutation().getRow());
+ if (row.equals("19059")) {
+ Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+ } else if (row.equals("59056")) {
+ Assert.assertEquals(Status.INVISIBLE_VISIBILITY, result.getStatus());
+ } else if (row.equals("99006")) {
+ Assert.assertEquals(Status.VIOLATED, result.getStatus());
+ } else if (row.equals("90909")) {
+ Assert.assertEquals(Status.REJECTED, result.getStatus());
+ }
+ rows.add(row);
+ }
+
+ Assert.assertEquals(4, rows.size());
+
+ Scanner scanner = conn.createScanner(table, new Authorizations("A"));
+ scanner.fetchColumn(new Text("tx"), new Text("seq"));
+
+ Iterator<Entry<Key,Value>> iter = scanner.iterator();
+ Assert.assertEquals("1", iter.next().getValue().toString());
+ Assert.assertFalse(iter.hasNext());
+
+ cw.close();
+ }
+
+ @Test
+ public void testSameRow() throws Exception {
+ // test multiple mutations for same row in same batch
+
+ String table = "foo8";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+ ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+ cm1.put("tx", "seq", "1");
+ cm1.put("data", "x", "a");
+
+ Assert.assertEquals(Status.ACCEPTED, cw.write(cm1).getStatus());
+
+ ConditionalMutation cm2 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+ cm2.put("tx", "seq", "2");
+ cm2.put("data", "x", "b");
+
+ ConditionalMutation cm3 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+ cm3.put("tx", "seq", "2");
+ cm3.put("data", "x", "c");
+
+ ConditionalMutation cm4 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+ cm4.put("tx", "seq", "2");
+ cm4.put("data", "x", "d");
+
+ Iterator<Result> results = cw.write(Arrays.asList(cm2, cm3, cm4).iterator());
+
+ int accepted = 0;
+ int rejected = 0;
+ int total = 0;
+
+ while (results.hasNext()) {
+ Status status = results.next().getStatus();
+ if (status == Status.ACCEPTED)
+ accepted++;
+ if (status == Status.REJECTED)
+ rejected++;
+ total++;
+ }
+
+ Assert.assertEquals(1, accepted);
+ Assert.assertEquals(2, rejected);
+ Assert.assertEquals(3, total);
+
+ cw.close();
+ }
+
+ private static class Stats {
+
+ ByteSequence row = null;
+ int seq;
+ long sum;
+ int data[] = new int[10];
+
+ public Stats(Iterator<Entry<Key,Value>> iterator) {
+ while (iterator.hasNext()) {
+ Entry<Key,Value> entry = iterator.next();
+
+ if (row == null)
+ row = entry.getKey().getRowData();
+
+ String cf = entry.getKey().getColumnFamilyData().toString();
+ String cq = entry.getKey().getColumnQualifierData().toString();
+
+ if (cf.equals("data")) {
+ data[Integer.parseInt(cq)] = Integer.parseInt(entry.getValue().toString());
+ } else if (cf.equals("meta")) {
+ if (cq.equals("sum")) {
+ sum = Long.parseLong(entry.getValue().toString());
+ } else if (cq.equals("seq")) {
+ seq = Integer.parseInt(entry.getValue().toString());
+ }
+ }
+ }
+
+ long sum2 = 0;
+
+ for (int datum : data) {
+ sum2 += datum;
+ }
+
+ Assert.assertEquals(sum2, sum);
+ }
+
+ public Stats(ByteSequence row) {
+ this.row = row;
+ for (int i = 0; i < data.length; i++) {
+ this.data[i] = 0;
+ }
+ this.seq = -1;
+ this.sum = 0;
+ }
+
+ void set(int index, int value) {
+ sum -= data[index];
+ sum += value;
+ data[index] = value;
+ }
+
+ ConditionalMutation toMutation() {
+ Condition cond = new Condition("meta", "seq");
+ if (seq >= 0)
+ cond.setValue(seq + "");
+
+ ConditionalMutation cm = new ConditionalMutation(row, cond);
+
+ cm.put("meta", "seq", (seq + 1) + "");
+ cm.put("meta", "sum", (sum) + "");
+
+ for (int i = 0; i < data.length; i++) {
+ cm.put("data", i + "", data[i] + "");
+ }
+
+ return cm;
+ }
+
+ public String toString() {
+ return row + " " + seq + " " + sum;
+ }
+ }
+
+ private static class MutatorTask implements Runnable {
+ String table;
+ ArrayList<ByteSequence> rows;
+ ConditionalWriter cw;
+ Connector conn;
+ AtomicBoolean failed;
+
+ public MutatorTask(String table, Connector conn, ArrayList<ByteSequence> rows, ConditionalWriter cw, AtomicBoolean failed) {
+ this.table = table;
+ this.rows = rows;
+ this.conn = conn;
+ this.cw = cw;
+ this.failed = failed;
+ }
+
+ @Override
+ public void run() {
+ try {
+ Random rand = new Random();
+
+ Scanner scanner = new IsolatedScanner(conn.createScanner(table, Authorizations.EMPTY));
+
+ for (int i = 0; i < 20; i++) {
+ int numRows = rand.nextInt(10) + 1;
+
+ ArrayList<ByteSequence> changes = new ArrayList<ByteSequence>(numRows);
+ ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+
+ for (int j = 0; j < numRows; j++)
+ changes.add(rows.get(rand.nextInt(rows.size())));
+
+ for (ByteSequence row : changes) {
+ scanner.setRange(new Range(row.toString()));
+ Stats stats = new Stats(scanner.iterator());
+ stats.set(rand.nextInt(10), Math.abs(rand.nextInt()));
+ mutations.add(stats.toMutation());
+ }
+
+ ArrayList<ByteSequence> changed = new ArrayList<ByteSequence>(numRows);
+ Iterator<Result> results = cw.write(mutations.iterator());
+ while (results.hasNext()) {
+ Result result = results.next();
+ changed.add(new ArrayByteSequence(result.getMutation().getRow()));
+ }
+
+ Collections.sort(changes);
+ Collections.sort(changed);
+
+ Assert.assertEquals(changes, changed);
+
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ failed.set(true);
+ }
+ }
+ }
+
+ @Test
+ public void testThreads() throws Exception {
+ // test multiple threads using a single conditional writer
+
+ String table = "foo9";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+
+ Random rand = new Random();
+
+ switch (rand.nextInt(3)) {
+ case 1:
+ conn.tableOperations().addSplits(table, nss("4"));
+ break;
+ case 2:
+ conn.tableOperations().addSplits(table, nss("3", "5"));
+ break;
+ }
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+ ArrayList<ByteSequence> rows = new ArrayList<ByteSequence>();
+
+
+ for (int i = 0; i < 1000; i++) {
+ rows.add(new ArrayByteSequence(FastFormat.toZeroPaddedString(Math.abs(rand.nextLong()), 16, 16, new byte[0])));
+ }
+
+ ArrayList<ConditionalMutation> mutations = new ArrayList<ConditionalMutation>();
+
+ for (ByteSequence row : rows)
+ mutations.add(new Stats(row).toMutation());
+
+ ArrayList<ByteSequence> rows2 = new ArrayList<ByteSequence>();
+ Iterator<Result> results = cw.write(mutations.iterator());
+ while (results.hasNext()) {
+ Result result = results.next();
+ Assert.assertEquals(Status.ACCEPTED, result.getStatus());
+ rows2.add(new ArrayByteSequence(result.getMutation().getRow()));
+ }
+
+ Collections.sort(rows);
+ Collections.sort(rows2);
+
+ Assert.assertEquals(rows, rows2);
+
+ AtomicBoolean failed = new AtomicBoolean(false);
+
+ ExecutorService tp = Executors.newFixedThreadPool(5);
+ for (int i = 0; i < 5; i++) {
+ tp.submit(new MutatorTask(table, conn, rows, cw, failed));
+ }
+
+ tp.shutdown();
+
+ while (!tp.isTerminated()) {
+ tp.awaitTermination(1, TimeUnit.MINUTES);
+ }
+
+ Assert.assertFalse(failed.get());
+
+ Scanner scanner = conn.createScanner(table, Authorizations.EMPTY);
+
+ RowIterator rowIter = new RowIterator(scanner);
+
+ while (rowIter.hasNext()) {
+ Iterator<Entry<Key,Value>> row = rowIter.next();
+ new Stats(row);
+ }
+ }
+
+ private SortedSet<Text> nss(String... splits) {
+ TreeSet<Text> ret = new TreeSet<Text>();
+ for (String split : splits)
+ ret.add(new Text(split));
+
+ return ret;
+ }
+
+ @Test
+ public void testSecurity() throws Exception {
+ // test against table user does not have read and/or write permissions for
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.securityOperations().createLocalUser("user1", new PasswordToken("u1p"));
+
+ conn.tableOperations().create("sect1");
+ conn.tableOperations().create("sect2");
+ conn.tableOperations().create("sect3");
+
+ conn.securityOperations().grantTablePermission("user1", "sect1", TablePermission.READ);
+ conn.securityOperations().grantTablePermission("user1", "sect2", TablePermission.WRITE);
+ conn.securityOperations().grantTablePermission("user1", "sect3", TablePermission.READ);
+ conn.securityOperations().grantTablePermission("user1", "sect3", TablePermission.WRITE);
+
+ Connector conn2 = zki.getConnector("user1", new PasswordToken("u1p"));
+
+ ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+ cm1.put("tx", "seq", "1");
+ cm1.put("data", "x", "a");
+
+ ConditionalWriter cw1 = conn2.createConditionalWriter("sect1", new ConditionalWriterConfig());
+ ConditionalWriter cw2 = conn2.createConditionalWriter("sect2", new ConditionalWriterConfig());
+ ConditionalWriter cw3 = conn2.createConditionalWriter("sect3", new ConditionalWriterConfig());
+
+ Assert.assertEquals(Status.ACCEPTED, cw3.write(cm1).getStatus());
+
+ try {
+ cw1.write(cm1).getStatus();
+ Assert.assertFalse(true);
+ } catch (AccumuloSecurityException ase) {
+
+ }
+
+ try {
+ cw2.write(cm1).getStatus();
+ Assert.assertFalse(true);
+ } catch (AccumuloSecurityException ase) {
+
+ }
+
+ }
+
+
+ @Test
+ public void testTimeout() throws Exception {
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ String table = "fooT";
+
+ conn.tableOperations().create(table);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig().setTimeout(1, TimeUnit.SECONDS));
+
+ ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+ cm1.put("tx", "seq", "1");
+ cm1.put("data", "x", "a");
+
+ Assert.assertEquals(cw.write(cm1).getStatus(), Status.ACCEPTED);
+
+ IteratorSetting is = new IteratorSetting(5, SlowIterator.class);
+ SlowIterator.setSeekSleepTime(is, 1500);
+
+ ConditionalMutation cm2 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1").setIterators(is));
+ cm2.put("tx", "seq", "2");
+ cm2.put("data", "x", "b");
+
+ Assert.assertEquals(cw.write(cm2).getStatus(), Status.UNKNOWN);
+
+ Scanner scanner = conn.createScanner(table, Authorizations.EMPTY);
+
+ for (Entry<Key,Value> entry : scanner) {
+ String cf = entry.getKey().getColumnFamilyData().toString();
+ String cq = entry.getKey().getColumnQualifierData().toString();
+ String val = entry.getValue().toString();
+
+ if (cf.equals("tx") && cq.equals("seq"))
+ Assert.assertEquals("1", val);
+ else if (cf.equals("data") && cq.equals("x"))
+ Assert.assertEquals("a", val);
+ else
+ Assert.assertTrue(false);
+ }
+
+ ConditionalMutation cm3 = new ConditionalMutation("r1", new Condition("tx", "seq").setValue("1"));
+ cm3.put("tx", "seq", "2");
+ cm3.put("data", "x", "b");
+
+ Assert.assertEquals(cw.write(cm3).getStatus(), Status.ACCEPTED);
+
+ cw.close();
+
+ }
+
+ @Test
+ public void testDeleteTable() throws Exception {
+ String table = "foo12";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ try {
+ conn.createConditionalWriter(table, new ConditionalWriterConfig());
+ Assert.assertFalse(true);
+ } catch (TableNotFoundException e) {}
+
+ conn.tableOperations().create(table);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+ conn.tableOperations().delete(table);
+
+ ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+ cm1.put("tx", "seq", "1");
+ cm1.put("data", "x", "a");
+
+ Result result = cw.write(cm1);
+
+ try {
+ result.getStatus();
+ Assert.assertFalse(true);
+ } catch (AccumuloException ae) {
+ Assert.assertEquals(TableDeletedException.class, ae.getCause().getClass());
+ }
+
+ }
+
+ @Test
+ public void testOffline() throws Exception {
+ String table = "foo11";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+ conn.tableOperations().offline(table);
+
+ waitForSingleTabletTableToGoOffline(table, zki);
+
+ ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
+ cm1.put("tx", "seq", "1");
+ cm1.put("data", "x", "a");
+
+ Result result = cw.write(cm1);
+
+ try {
+ result.getStatus();
+ Assert.assertFalse(true);
+ } catch (AccumuloException ae) {
+ Assert.assertEquals(TableOfflineException.class, ae.getCause().getClass());
+ }
+
+ cw.close();
+
+ try {
+ conn.createConditionalWriter(table, new ConditionalWriterConfig());
+ Assert.assertFalse(true);
+ } catch (TableOfflineException e) {}
+ }
+
+ void waitForSingleTabletTableToGoOffline(String table, ZooKeeperInstance zki) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+ TabletLocator locator = TabletLocator.getLocator(zki, new Text(Tables.getNameToIdMap(zki).get(table)));
+ while (locator.locateTablet(new Text("a"), false, false, CredentialHelper.create("root", new PasswordToken(secret), zki.getInstanceID())) != null) {
+ UtilWaitThread.sleep(50);
+ locator.invalidateCache();
+ }
+ }
+
+ @Test
+ public void testError() throws Exception {
+ String table = "foo10";
+
+ ZooKeeperInstance zki = new ZooKeeperInstance(cluster.getInstanceName(), cluster.getZooKeepers());
+ Connector conn = zki.getConnector("root", new PasswordToken(secret));
+
+ conn.tableOperations().create(table);
+
+ ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
+
+ IteratorSetting iterSetting = new IteratorSetting(5, BadIterator.class);
+
+ ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq").setIterators(iterSetting));
+ cm1.put("tx", "seq", "1");
+ cm1.put("data", "x", "a");
+
+ Result result = cw.write(cm1);
+
+ try {
+ result.getStatus();
+ Assert.assertFalse(true);
+ } catch (AccumuloException ae) {
+
+ }
+
+ cw.close();
+
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ cluster.stop();
+ // folder.delete();
+ }
+}
[5/6] ACCUMULO-1000 added conditional mutations to Accumulo
Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java
new file mode 100644
index 0000000..fe72fd6
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMResult.java
@@ -0,0 +1,516 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class TCMResult implements org.apache.thrift.TBase<TCMResult, TCMResult._Fields>, java.io.Serializable, Cloneable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCMResult");
+
+ private static final org.apache.thrift.protocol.TField CMID_FIELD_DESC = new org.apache.thrift.protocol.TField("cmid", org.apache.thrift.protocol.TType.I64, (short)1);
+ private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.I32, (short)2);
+
+ private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TCMResultStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TCMResultTupleSchemeFactory());
+ }
+
+ public long cmid; // required
+ /**
+ *
+ * @see TCMStatus
+ */
+ public TCMStatus status; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ CMID((short)1, "cmid"),
+ /**
+ *
+ * @see TCMStatus
+ */
+ STATUS((short)2, "status");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // CMID
+ return CMID;
+ case 2: // STATUS
+ return STATUS;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __CMID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.CMID, new org.apache.thrift.meta_data.FieldMetaData("cmid", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+ tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TCMStatus.class)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCMResult.class, metaDataMap);
+ }
+
+ public TCMResult() {
+ }
+
+ public TCMResult(
+ long cmid,
+ TCMStatus status)
+ {
+ this();
+ this.cmid = cmid;
+ setCmidIsSet(true);
+ this.status = status;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public TCMResult(TCMResult other) {
+ __isset_bitfield = other.__isset_bitfield;
+ this.cmid = other.cmid;
+ if (other.isSetStatus()) {
+ this.status = other.status;
+ }
+ }
+
+ public TCMResult deepCopy() {
+ return new TCMResult(this);
+ }
+
+ @Override
+ public void clear() {
+ setCmidIsSet(false);
+ this.cmid = 0;
+ this.status = null;
+ }
+
+ public long getCmid() {
+ return this.cmid;
+ }
+
+ public TCMResult setCmid(long cmid) {
+ this.cmid = cmid;
+ setCmidIsSet(true);
+ return this;
+ }
+
+ public void unsetCmid() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CMID_ISSET_ID);
+ }
+
+ /** Returns true if field cmid is set (has been assigned a value) and false otherwise */
+ public boolean isSetCmid() {
+ return EncodingUtils.testBit(__isset_bitfield, __CMID_ISSET_ID);
+ }
+
+ public void setCmidIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CMID_ISSET_ID, value);
+ }
+
+ /**
+ *
+ * @see TCMStatus
+ */
+ public TCMStatus getStatus() {
+ return this.status;
+ }
+
+ /**
+ *
+ * @see TCMStatus
+ */
+ public TCMResult setStatus(TCMStatus status) {
+ this.status = status;
+ return this;
+ }
+
+ public void unsetStatus() {
+ this.status = null;
+ }
+
+ /** Returns true if field status is set (has been assigned a value) and false otherwise */
+ public boolean isSetStatus() {
+ return this.status != null;
+ }
+
+ public void setStatusIsSet(boolean value) {
+ if (!value) {
+ this.status = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case CMID:
+ if (value == null) {
+ unsetCmid();
+ } else {
+ setCmid((Long)value);
+ }
+ break;
+
+ case STATUS:
+ if (value == null) {
+ unsetStatus();
+ } else {
+ setStatus((TCMStatus)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case CMID:
+ return Long.valueOf(getCmid());
+
+ case STATUS:
+ return getStatus();
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case CMID:
+ return isSetCmid();
+ case STATUS:
+ return isSetStatus();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TCMResult)
+ return this.equals((TCMResult)that);
+ return false;
+ }
+
+ public boolean equals(TCMResult that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_cmid = true;
+ boolean that_present_cmid = true;
+ if (this_present_cmid || that_present_cmid) {
+ if (!(this_present_cmid && that_present_cmid))
+ return false;
+ if (this.cmid != that.cmid)
+ return false;
+ }
+
+ boolean this_present_status = true && this.isSetStatus();
+ boolean that_present_status = true && that.isSetStatus();
+ if (this_present_status || that_present_status) {
+ if (!(this_present_status && that_present_status))
+ return false;
+ if (!this.status.equals(that.status))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(TCMResult other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ TCMResult typedOther = (TCMResult)other;
+
+ lastComparison = Boolean.valueOf(isSetCmid()).compareTo(typedOther.isSetCmid());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCmid()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cmid, typedOther.cmid);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetStatus()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("TCMResult(");
+ boolean first = true;
+
+ sb.append("cmid:");
+ sb.append(this.cmid);
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("status:");
+ if (this.status == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.status);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TCMResultStandardSchemeFactory implements SchemeFactory {
+ public TCMResultStandardScheme getScheme() {
+ return new TCMResultStandardScheme();
+ }
+ }
+
+ private static class TCMResultStandardScheme extends StandardScheme<TCMResult> {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TCMResult struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // CMID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.cmid = iprot.readI64();
+ struct.setCmidIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // STATUS
+ if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+ struct.status = TCMStatus.findByValue(iprot.readI32());
+ struct.setStatusIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot, TCMResult struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ oprot.writeFieldBegin(CMID_FIELD_DESC);
+ oprot.writeI64(struct.cmid);
+ oprot.writeFieldEnd();
+ if (struct.status != null) {
+ oprot.writeFieldBegin(STATUS_FIELD_DESC);
+ oprot.writeI32(struct.status.getValue());
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TCMResultTupleSchemeFactory implements SchemeFactory {
+ public TCMResultTupleScheme getScheme() {
+ return new TCMResultTupleScheme();
+ }
+ }
+
+ private static class TCMResultTupleScheme extends TupleScheme<TCMResult> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TCMResult struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetCmid()) {
+ optionals.set(0);
+ }
+ if (struct.isSetStatus()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetCmid()) {
+ oprot.writeI64(struct.cmid);
+ }
+ if (struct.isSetStatus()) {
+ oprot.writeI32(struct.status.getValue());
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TCMResult struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.cmid = iprot.readI64();
+ struct.setCmidIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.status = TCMStatus.findByValue(iprot.readI32());
+ struct.setStatusIsSet(true);
+ }
+ }
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java
new file mode 100644
index 0000000..6ef94d5
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCMStatus.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+@SuppressWarnings("all") public enum TCMStatus implements org.apache.thrift.TEnum {
+ ACCEPTED(0),
+ REJECTED(1),
+ VIOLATED(2),
+ IGNORED(3);
+
+ private final int value;
+
+ private TCMStatus(int value) {
+ this.value = value;
+ }
+
+ /**
+ * Get the integer value of this enum value, as defined in the Thrift IDL.
+ */
+ public int getValue() {
+ return value;
+ }
+
+ /**
+ * Find a the enum type by its integer value, as defined in the Thrift IDL.
+ * @return null if the value is not found.
+ */
+ public static TCMStatus findByValue(int value) {
+ switch (value) {
+ case 0:
+ return ACCEPTED;
+ case 1:
+ return REJECTED;
+ case 2:
+ return VIOLATED;
+ case 3:
+ return IGNORED;
+ default:
+ return null;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java
new file mode 100644
index 0000000..40a12be
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TCondition.java
@@ -0,0 +1,1049 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class TCondition implements org.apache.thrift.TBase<TCondition, TCondition._Fields>, java.io.Serializable, Cloneable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCondition");
+
+ private static final org.apache.thrift.protocol.TField CF_FIELD_DESC = new org.apache.thrift.protocol.TField("cf", org.apache.thrift.protocol.TType.STRING, (short)1);
+ private static final org.apache.thrift.protocol.TField CQ_FIELD_DESC = new org.apache.thrift.protocol.TField("cq", org.apache.thrift.protocol.TType.STRING, (short)2);
+ private static final org.apache.thrift.protocol.TField CV_FIELD_DESC = new org.apache.thrift.protocol.TField("cv", org.apache.thrift.protocol.TType.STRING, (short)3);
+ private static final org.apache.thrift.protocol.TField TS_FIELD_DESC = new org.apache.thrift.protocol.TField("ts", org.apache.thrift.protocol.TType.I64, (short)4);
+ private static final org.apache.thrift.protocol.TField HAS_TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("hasTimestamp", org.apache.thrift.protocol.TType.BOOL, (short)5);
+ private static final org.apache.thrift.protocol.TField VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("val", org.apache.thrift.protocol.TType.STRING, (short)6);
+ private static final org.apache.thrift.protocol.TField ITERATORS_FIELD_DESC = new org.apache.thrift.protocol.TField("iterators", org.apache.thrift.protocol.TType.STRING, (short)7);
+
+ private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TConditionStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TConditionTupleSchemeFactory());
+ }
+
+ public ByteBuffer cf; // required
+ public ByteBuffer cq; // required
+ public ByteBuffer cv; // required
+ public long ts; // required
+ public boolean hasTimestamp; // required
+ public ByteBuffer val; // required
+ public ByteBuffer iterators; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ CF((short)1, "cf"),
+ CQ((short)2, "cq"),
+ CV((short)3, "cv"),
+ TS((short)4, "ts"),
+ HAS_TIMESTAMP((short)5, "hasTimestamp"),
+ VAL((short)6, "val"),
+ ITERATORS((short)7, "iterators");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // CF
+ return CF;
+ case 2: // CQ
+ return CQ;
+ case 3: // CV
+ return CV;
+ case 4: // TS
+ return TS;
+ case 5: // HAS_TIMESTAMP
+ return HAS_TIMESTAMP;
+ case 6: // VAL
+ return VAL;
+ case 7: // ITERATORS
+ return ITERATORS;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __TS_ISSET_ID = 0;
+ private static final int __HASTIMESTAMP_ISSET_ID = 1;
+ private byte __isset_bitfield = 0;
+ public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.CF, new org.apache.thrift.meta_data.FieldMetaData("cf", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ tmpMap.put(_Fields.CQ, new org.apache.thrift.meta_data.FieldMetaData("cq", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ tmpMap.put(_Fields.CV, new org.apache.thrift.meta_data.FieldMetaData("cv", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ tmpMap.put(_Fields.TS, new org.apache.thrift.meta_data.FieldMetaData("ts", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+ tmpMap.put(_Fields.HAS_TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("hasTimestamp", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+ tmpMap.put(_Fields.VAL, new org.apache.thrift.meta_data.FieldMetaData("val", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ tmpMap.put(_Fields.ITERATORS, new org.apache.thrift.meta_data.FieldMetaData("iterators", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCondition.class, metaDataMap);
+ }
+
+ public TCondition() {
+ }
+
+ public TCondition(
+ ByteBuffer cf,
+ ByteBuffer cq,
+ ByteBuffer cv,
+ long ts,
+ boolean hasTimestamp,
+ ByteBuffer val,
+ ByteBuffer iterators)
+ {
+ this();
+ this.cf = cf;
+ this.cq = cq;
+ this.cv = cv;
+ this.ts = ts;
+ setTsIsSet(true);
+ this.hasTimestamp = hasTimestamp;
+ setHasTimestampIsSet(true);
+ this.val = val;
+ this.iterators = iterators;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public TCondition(TCondition other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetCf()) {
+ this.cf = org.apache.thrift.TBaseHelper.copyBinary(other.cf);
+;
+ }
+ if (other.isSetCq()) {
+ this.cq = org.apache.thrift.TBaseHelper.copyBinary(other.cq);
+;
+ }
+ if (other.isSetCv()) {
+ this.cv = org.apache.thrift.TBaseHelper.copyBinary(other.cv);
+;
+ }
+ this.ts = other.ts;
+ this.hasTimestamp = other.hasTimestamp;
+ if (other.isSetVal()) {
+ this.val = org.apache.thrift.TBaseHelper.copyBinary(other.val);
+;
+ }
+ if (other.isSetIterators()) {
+ this.iterators = org.apache.thrift.TBaseHelper.copyBinary(other.iterators);
+;
+ }
+ }
+
+ public TCondition deepCopy() {
+ return new TCondition(this);
+ }
+
+ @Override
+ public void clear() {
+ this.cf = null;
+ this.cq = null;
+ this.cv = null;
+ setTsIsSet(false);
+ this.ts = 0;
+ setHasTimestampIsSet(false);
+ this.hasTimestamp = false;
+ this.val = null;
+ this.iterators = null;
+ }
+
+ public byte[] getCf() {
+ setCf(org.apache.thrift.TBaseHelper.rightSize(cf));
+ return cf == null ? null : cf.array();
+ }
+
+ public ByteBuffer bufferForCf() {
+ return cf;
+ }
+
+ public TCondition setCf(byte[] cf) {
+ setCf(cf == null ? (ByteBuffer)null : ByteBuffer.wrap(cf));
+ return this;
+ }
+
+ public TCondition setCf(ByteBuffer cf) {
+ this.cf = cf;
+ return this;
+ }
+
+ public void unsetCf() {
+ this.cf = null;
+ }
+
+ /** Returns true if field cf is set (has been assigned a value) and false otherwise */
+ public boolean isSetCf() {
+ return this.cf != null;
+ }
+
+ public void setCfIsSet(boolean value) {
+ if (!value) {
+ this.cf = null;
+ }
+ }
+
+ public byte[] getCq() {
+ setCq(org.apache.thrift.TBaseHelper.rightSize(cq));
+ return cq == null ? null : cq.array();
+ }
+
+ public ByteBuffer bufferForCq() {
+ return cq;
+ }
+
+ public TCondition setCq(byte[] cq) {
+ setCq(cq == null ? (ByteBuffer)null : ByteBuffer.wrap(cq));
+ return this;
+ }
+
+ public TCondition setCq(ByteBuffer cq) {
+ this.cq = cq;
+ return this;
+ }
+
+ public void unsetCq() {
+ this.cq = null;
+ }
+
+ /** Returns true if field cq is set (has been assigned a value) and false otherwise */
+ public boolean isSetCq() {
+ return this.cq != null;
+ }
+
+ public void setCqIsSet(boolean value) {
+ if (!value) {
+ this.cq = null;
+ }
+ }
+
+ public byte[] getCv() {
+ setCv(org.apache.thrift.TBaseHelper.rightSize(cv));
+ return cv == null ? null : cv.array();
+ }
+
+ public ByteBuffer bufferForCv() {
+ return cv;
+ }
+
+ public TCondition setCv(byte[] cv) {
+ setCv(cv == null ? (ByteBuffer)null : ByteBuffer.wrap(cv));
+ return this;
+ }
+
+ public TCondition setCv(ByteBuffer cv) {
+ this.cv = cv;
+ return this;
+ }
+
+ public void unsetCv() {
+ this.cv = null;
+ }
+
+ /** Returns true if field cv is set (has been assigned a value) and false otherwise */
+ public boolean isSetCv() {
+ return this.cv != null;
+ }
+
+ public void setCvIsSet(boolean value) {
+ if (!value) {
+ this.cv = null;
+ }
+ }
+
+ public long getTs() {
+ return this.ts;
+ }
+
+ public TCondition setTs(long ts) {
+ this.ts = ts;
+ setTsIsSet(true);
+ return this;
+ }
+
+ public void unsetTs() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TS_ISSET_ID);
+ }
+
+ /** Returns true if field ts is set (has been assigned a value) and false otherwise */
+ public boolean isSetTs() {
+ return EncodingUtils.testBit(__isset_bitfield, __TS_ISSET_ID);
+ }
+
+ public void setTsIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TS_ISSET_ID, value);
+ }
+
+ public boolean isHasTimestamp() {
+ return this.hasTimestamp;
+ }
+
+ public TCondition setHasTimestamp(boolean hasTimestamp) {
+ this.hasTimestamp = hasTimestamp;
+ setHasTimestampIsSet(true);
+ return this;
+ }
+
+ public void unsetHasTimestamp() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASTIMESTAMP_ISSET_ID);
+ }
+
+ /** Returns true if field hasTimestamp is set (has been assigned a value) and false otherwise */
+ public boolean isSetHasTimestamp() {
+ return EncodingUtils.testBit(__isset_bitfield, __HASTIMESTAMP_ISSET_ID);
+ }
+
+ public void setHasTimestampIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASTIMESTAMP_ISSET_ID, value);
+ }
+
+ public byte[] getVal() {
+ setVal(org.apache.thrift.TBaseHelper.rightSize(val));
+ return val == null ? null : val.array();
+ }
+
+ public ByteBuffer bufferForVal() {
+ return val;
+ }
+
+ public TCondition setVal(byte[] val) {
+ setVal(val == null ? (ByteBuffer)null : ByteBuffer.wrap(val));
+ return this;
+ }
+
+ public TCondition setVal(ByteBuffer val) {
+ this.val = val;
+ return this;
+ }
+
+ public void unsetVal() {
+ this.val = null;
+ }
+
+ /** Returns true if field val is set (has been assigned a value) and false otherwise */
+ public boolean isSetVal() {
+ return this.val != null;
+ }
+
+ public void setValIsSet(boolean value) {
+ if (!value) {
+ this.val = null;
+ }
+ }
+
+ public byte[] getIterators() {
+ setIterators(org.apache.thrift.TBaseHelper.rightSize(iterators));
+ return iterators == null ? null : iterators.array();
+ }
+
+ public ByteBuffer bufferForIterators() {
+ return iterators;
+ }
+
+ public TCondition setIterators(byte[] iterators) {
+ setIterators(iterators == null ? (ByteBuffer)null : ByteBuffer.wrap(iterators));
+ return this;
+ }
+
+ public TCondition setIterators(ByteBuffer iterators) {
+ this.iterators = iterators;
+ return this;
+ }
+
+ public void unsetIterators() {
+ this.iterators = null;
+ }
+
+ /** Returns true if field iterators is set (has been assigned a value) and false otherwise */
+ public boolean isSetIterators() {
+ return this.iterators != null;
+ }
+
+ public void setIteratorsIsSet(boolean value) {
+ if (!value) {
+ this.iterators = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case CF:
+ if (value == null) {
+ unsetCf();
+ } else {
+ setCf((ByteBuffer)value);
+ }
+ break;
+
+ case CQ:
+ if (value == null) {
+ unsetCq();
+ } else {
+ setCq((ByteBuffer)value);
+ }
+ break;
+
+ case CV:
+ if (value == null) {
+ unsetCv();
+ } else {
+ setCv((ByteBuffer)value);
+ }
+ break;
+
+ case TS:
+ if (value == null) {
+ unsetTs();
+ } else {
+ setTs((Long)value);
+ }
+ break;
+
+ case HAS_TIMESTAMP:
+ if (value == null) {
+ unsetHasTimestamp();
+ } else {
+ setHasTimestamp((Boolean)value);
+ }
+ break;
+
+ case VAL:
+ if (value == null) {
+ unsetVal();
+ } else {
+ setVal((ByteBuffer)value);
+ }
+ break;
+
+ case ITERATORS:
+ if (value == null) {
+ unsetIterators();
+ } else {
+ setIterators((ByteBuffer)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case CF:
+ return getCf();
+
+ case CQ:
+ return getCq();
+
+ case CV:
+ return getCv();
+
+ case TS:
+ return Long.valueOf(getTs());
+
+ case HAS_TIMESTAMP:
+ return Boolean.valueOf(isHasTimestamp());
+
+ case VAL:
+ return getVal();
+
+ case ITERATORS:
+ return getIterators();
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case CF:
+ return isSetCf();
+ case CQ:
+ return isSetCq();
+ case CV:
+ return isSetCv();
+ case TS:
+ return isSetTs();
+ case HAS_TIMESTAMP:
+ return isSetHasTimestamp();
+ case VAL:
+ return isSetVal();
+ case ITERATORS:
+ return isSetIterators();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TCondition)
+ return this.equals((TCondition)that);
+ return false;
+ }
+
+ public boolean equals(TCondition that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_cf = true && this.isSetCf();
+ boolean that_present_cf = true && that.isSetCf();
+ if (this_present_cf || that_present_cf) {
+ if (!(this_present_cf && that_present_cf))
+ return false;
+ if (!this.cf.equals(that.cf))
+ return false;
+ }
+
+ boolean this_present_cq = true && this.isSetCq();
+ boolean that_present_cq = true && that.isSetCq();
+ if (this_present_cq || that_present_cq) {
+ if (!(this_present_cq && that_present_cq))
+ return false;
+ if (!this.cq.equals(that.cq))
+ return false;
+ }
+
+ boolean this_present_cv = true && this.isSetCv();
+ boolean that_present_cv = true && that.isSetCv();
+ if (this_present_cv || that_present_cv) {
+ if (!(this_present_cv && that_present_cv))
+ return false;
+ if (!this.cv.equals(that.cv))
+ return false;
+ }
+
+ boolean this_present_ts = true;
+ boolean that_present_ts = true;
+ if (this_present_ts || that_present_ts) {
+ if (!(this_present_ts && that_present_ts))
+ return false;
+ if (this.ts != that.ts)
+ return false;
+ }
+
+ boolean this_present_hasTimestamp = true;
+ boolean that_present_hasTimestamp = true;
+ if (this_present_hasTimestamp || that_present_hasTimestamp) {
+ if (!(this_present_hasTimestamp && that_present_hasTimestamp))
+ return false;
+ if (this.hasTimestamp != that.hasTimestamp)
+ return false;
+ }
+
+ boolean this_present_val = true && this.isSetVal();
+ boolean that_present_val = true && that.isSetVal();
+ if (this_present_val || that_present_val) {
+ if (!(this_present_val && that_present_val))
+ return false;
+ if (!this.val.equals(that.val))
+ return false;
+ }
+
+ boolean this_present_iterators = true && this.isSetIterators();
+ boolean that_present_iterators = true && that.isSetIterators();
+ if (this_present_iterators || that_present_iterators) {
+ if (!(this_present_iterators && that_present_iterators))
+ return false;
+ if (!this.iterators.equals(that.iterators))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(TCondition other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ TCondition typedOther = (TCondition)other;
+
+ lastComparison = Boolean.valueOf(isSetCf()).compareTo(typedOther.isSetCf());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCf()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cf, typedOther.cf);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetCq()).compareTo(typedOther.isSetCq());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCq()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cq, typedOther.cq);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetCv()).compareTo(typedOther.isSetCv());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCv()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cv, typedOther.cv);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetTs()).compareTo(typedOther.isSetTs());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTs()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ts, typedOther.ts);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetHasTimestamp()).compareTo(typedOther.isSetHasTimestamp());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetHasTimestamp()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasTimestamp, typedOther.hasTimestamp);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetVal()).compareTo(typedOther.isSetVal());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetVal()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.val, typedOther.val);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetIterators()).compareTo(typedOther.isSetIterators());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetIterators()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.iterators, typedOther.iterators);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("TCondition(");
+ boolean first = true;
+
+ sb.append("cf:");
+ if (this.cf == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.cf, sb);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("cq:");
+ if (this.cq == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.cq, sb);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("cv:");
+ if (this.cv == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.cv, sb);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("ts:");
+ sb.append(this.ts);
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("hasTimestamp:");
+ sb.append(this.hasTimestamp);
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("val:");
+ if (this.val == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.val, sb);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("iterators:");
+ if (this.iterators == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.iterators, sb);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TConditionStandardSchemeFactory implements SchemeFactory {
+ public TConditionStandardScheme getScheme() {
+ return new TConditionStandardScheme();
+ }
+ }
+
+ private static class TConditionStandardScheme extends StandardScheme<TCondition> {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TCondition struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // CF
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.cf = iprot.readBinary();
+ struct.setCfIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // CQ
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.cq = iprot.readBinary();
+ struct.setCqIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // CV
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.cv = iprot.readBinary();
+ struct.setCvIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // TS
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.ts = iprot.readI64();
+ struct.setTsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 5: // HAS_TIMESTAMP
+ if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+ struct.hasTimestamp = iprot.readBool();
+ struct.setHasTimestampIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 6: // VAL
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.val = iprot.readBinary();
+ struct.setValIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 7: // ITERATORS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.iterators = iprot.readBinary();
+ struct.setIteratorsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot, TCondition struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.cf != null) {
+ oprot.writeFieldBegin(CF_FIELD_DESC);
+ oprot.writeBinary(struct.cf);
+ oprot.writeFieldEnd();
+ }
+ if (struct.cq != null) {
+ oprot.writeFieldBegin(CQ_FIELD_DESC);
+ oprot.writeBinary(struct.cq);
+ oprot.writeFieldEnd();
+ }
+ if (struct.cv != null) {
+ oprot.writeFieldBegin(CV_FIELD_DESC);
+ oprot.writeBinary(struct.cv);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(TS_FIELD_DESC);
+ oprot.writeI64(struct.ts);
+ oprot.writeFieldEnd();
+ oprot.writeFieldBegin(HAS_TIMESTAMP_FIELD_DESC);
+ oprot.writeBool(struct.hasTimestamp);
+ oprot.writeFieldEnd();
+ if (struct.val != null) {
+ oprot.writeFieldBegin(VAL_FIELD_DESC);
+ oprot.writeBinary(struct.val);
+ oprot.writeFieldEnd();
+ }
+ if (struct.iterators != null) {
+ oprot.writeFieldBegin(ITERATORS_FIELD_DESC);
+ oprot.writeBinary(struct.iterators);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TConditionTupleSchemeFactory implements SchemeFactory {
+ public TConditionTupleScheme getScheme() {
+ return new TConditionTupleScheme();
+ }
+ }
+
+ private static class TConditionTupleScheme extends TupleScheme<TCondition> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TCondition struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetCf()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCq()) {
+ optionals.set(1);
+ }
+ if (struct.isSetCv()) {
+ optionals.set(2);
+ }
+ if (struct.isSetTs()) {
+ optionals.set(3);
+ }
+ if (struct.isSetHasTimestamp()) {
+ optionals.set(4);
+ }
+ if (struct.isSetVal()) {
+ optionals.set(5);
+ }
+ if (struct.isSetIterators()) {
+ optionals.set(6);
+ }
+ oprot.writeBitSet(optionals, 7);
+ if (struct.isSetCf()) {
+ oprot.writeBinary(struct.cf);
+ }
+ if (struct.isSetCq()) {
+ oprot.writeBinary(struct.cq);
+ }
+ if (struct.isSetCv()) {
+ oprot.writeBinary(struct.cv);
+ }
+ if (struct.isSetTs()) {
+ oprot.writeI64(struct.ts);
+ }
+ if (struct.isSetHasTimestamp()) {
+ oprot.writeBool(struct.hasTimestamp);
+ }
+ if (struct.isSetVal()) {
+ oprot.writeBinary(struct.val);
+ }
+ if (struct.isSetIterators()) {
+ oprot.writeBinary(struct.iterators);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TCondition struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(7);
+ if (incoming.get(0)) {
+ struct.cf = iprot.readBinary();
+ struct.setCfIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.cq = iprot.readBinary();
+ struct.setCqIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.cv = iprot.readBinary();
+ struct.setCvIsSet(true);
+ }
+ if (incoming.get(3)) {
+ struct.ts = iprot.readI64();
+ struct.setTsIsSet(true);
+ }
+ if (incoming.get(4)) {
+ struct.hasTimestamp = iprot.readBool();
+ struct.setHasTimestampIsSet(true);
+ }
+ if (incoming.get(5)) {
+ struct.val = iprot.readBinary();
+ struct.setValIsSet(true);
+ }
+ if (incoming.get(6)) {
+ struct.iterators = iprot.readBinary();
+ struct.setIteratorsIsSet(true);
+ }
+ }
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java
new file mode 100644
index 0000000..1928803
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalMutation.java
@@ -0,0 +1,659 @@
+/*
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("all") public class TConditionalMutation implements org.apache.thrift.TBase<TConditionalMutation, TConditionalMutation._Fields>, java.io.Serializable, Cloneable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TConditionalMutation");
+
+ private static final org.apache.thrift.protocol.TField CONDITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("conditions", org.apache.thrift.protocol.TType.LIST, (short)1);
+ private static final org.apache.thrift.protocol.TField MUTATION_FIELD_DESC = new org.apache.thrift.protocol.TField("mutation", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)3);
+
+ private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TConditionalMutationStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TConditionalMutationTupleSchemeFactory());
+ }
+
+ public List<TCondition> conditions; // required
+ public TMutation mutation; // required
+ public long id; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ CONDITIONS((short)1, "conditions"),
+ MUTATION((short)2, "mutation"),
+ ID((short)3, "id");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // CONDITIONS
+ return CONDITIONS;
+ case 2: // MUTATION
+ return MUTATION;
+ case 3: // ID
+ return ID;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __ID_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.CONDITIONS, new org.apache.thrift.meta_data.FieldMetaData("conditions", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCondition.class))));
+ tmpMap.put(_Fields.MUTATION, new org.apache.thrift.meta_data.FieldMetaData("mutation", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TMutation.class)));
+ tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TConditionalMutation.class, metaDataMap);
+ }
+
+ public TConditionalMutation() {
+ }
+
+ public TConditionalMutation(
+ List<TCondition> conditions,
+ TMutation mutation,
+ long id)
+ {
+ this();
+ this.conditions = conditions;
+ this.mutation = mutation;
+ this.id = id;
+ setIdIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public TConditionalMutation(TConditionalMutation other) {
+ __isset_bitfield = other.__isset_bitfield;
+ if (other.isSetConditions()) {
+ List<TCondition> __this__conditions = new ArrayList<TCondition>();
+ for (TCondition other_element : other.conditions) {
+ __this__conditions.add(new TCondition(other_element));
+ }
+ this.conditions = __this__conditions;
+ }
+ if (other.isSetMutation()) {
+ this.mutation = new TMutation(other.mutation);
+ }
+ this.id = other.id;
+ }
+
+ public TConditionalMutation deepCopy() {
+ return new TConditionalMutation(this);
+ }
+
+ @Override
+ public void clear() {
+ this.conditions = null;
+ this.mutation = null;
+ setIdIsSet(false);
+ this.id = 0;
+ }
+
+ public int getConditionsSize() {
+ return (this.conditions == null) ? 0 : this.conditions.size();
+ }
+
+ public java.util.Iterator<TCondition> getConditionsIterator() {
+ return (this.conditions == null) ? null : this.conditions.iterator();
+ }
+
+ public void addToConditions(TCondition elem) {
+ if (this.conditions == null) {
+ this.conditions = new ArrayList<TCondition>();
+ }
+ this.conditions.add(elem);
+ }
+
+ public List<TCondition> getConditions() {
+ return this.conditions;
+ }
+
+ public TConditionalMutation setConditions(List<TCondition> conditions) {
+ this.conditions = conditions;
+ return this;
+ }
+
+ public void unsetConditions() {
+ this.conditions = null;
+ }
+
+ /** Returns true if field conditions is set (has been assigned a value) and false otherwise */
+ public boolean isSetConditions() {
+ return this.conditions != null;
+ }
+
+ public void setConditionsIsSet(boolean value) {
+ if (!value) {
+ this.conditions = null;
+ }
+ }
+
+ public TMutation getMutation() {
+ return this.mutation;
+ }
+
+ public TConditionalMutation setMutation(TMutation mutation) {
+ this.mutation = mutation;
+ return this;
+ }
+
+ public void unsetMutation() {
+ this.mutation = null;
+ }
+
+ /** Returns true if field mutation is set (has been assigned a value) and false otherwise */
+ public boolean isSetMutation() {
+ return this.mutation != null;
+ }
+
+ public void setMutationIsSet(boolean value) {
+ if (!value) {
+ this.mutation = null;
+ }
+ }
+
+ public long getId() {
+ return this.id;
+ }
+
+ public TConditionalMutation setId(long id) {
+ this.id = id;
+ setIdIsSet(true);
+ return this;
+ }
+
+ public void unsetId() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ID_ISSET_ID);
+ }
+
+ /** Returns true if field id is set (has been assigned a value) and false otherwise */
+ public boolean isSetId() {
+ return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID);
+ }
+
+ public void setIdIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value);
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case CONDITIONS:
+ if (value == null) {
+ unsetConditions();
+ } else {
+ setConditions((List<TCondition>)value);
+ }
+ break;
+
+ case MUTATION:
+ if (value == null) {
+ unsetMutation();
+ } else {
+ setMutation((TMutation)value);
+ }
+ break;
+
+ case ID:
+ if (value == null) {
+ unsetId();
+ } else {
+ setId((Long)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case CONDITIONS:
+ return getConditions();
+
+ case MUTATION:
+ return getMutation();
+
+ case ID:
+ return Long.valueOf(getId());
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case CONDITIONS:
+ return isSetConditions();
+ case MUTATION:
+ return isSetMutation();
+ case ID:
+ return isSetId();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TConditionalMutation)
+ return this.equals((TConditionalMutation)that);
+ return false;
+ }
+
+ public boolean equals(TConditionalMutation that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_conditions = true && this.isSetConditions();
+ boolean that_present_conditions = true && that.isSetConditions();
+ if (this_present_conditions || that_present_conditions) {
+ if (!(this_present_conditions && that_present_conditions))
+ return false;
+ if (!this.conditions.equals(that.conditions))
+ return false;
+ }
+
+ boolean this_present_mutation = true && this.isSetMutation();
+ boolean that_present_mutation = true && that.isSetMutation();
+ if (this_present_mutation || that_present_mutation) {
+ if (!(this_present_mutation && that_present_mutation))
+ return false;
+ if (!this.mutation.equals(that.mutation))
+ return false;
+ }
+
+ boolean this_present_id = true;
+ boolean that_present_id = true;
+ if (this_present_id || that_present_id) {
+ if (!(this_present_id && that_present_id))
+ return false;
+ if (this.id != that.id)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(TConditionalMutation other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ TConditionalMutation typedOther = (TConditionalMutation)other;
+
+ lastComparison = Boolean.valueOf(isSetConditions()).compareTo(typedOther.isSetConditions());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetConditions()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.conditions, typedOther.conditions);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetMutation()).compareTo(typedOther.isSetMutation());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetMutation()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutation, typedOther.mutation);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetId()).compareTo(typedOther.isSetId());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetId()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("TConditionalMutation(");
+ boolean first = true;
+
+ sb.append("conditions:");
+ if (this.conditions == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.conditions);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("mutation:");
+ if (this.mutation == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.mutation);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("id:");
+ sb.append(this.id);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (mutation != null) {
+ mutation.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TConditionalMutationStandardSchemeFactory implements SchemeFactory {
+ public TConditionalMutationStandardScheme getScheme() {
+ return new TConditionalMutationStandardScheme();
+ }
+ }
+
+ private static class TConditionalMutationStandardScheme extends StandardScheme<TConditionalMutation> {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TConditionalMutation struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // CONDITIONS
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list106 = iprot.readListBegin();
+ struct.conditions = new ArrayList<TCondition>(_list106.size);
+ for (int _i107 = 0; _i107 < _list106.size; ++_i107)
+ {
+ TCondition _elem108; // required
+ _elem108 = new TCondition();
+ _elem108.read(iprot);
+ struct.conditions.add(_elem108);
+ }
+ iprot.readListEnd();
+ }
+ struct.setConditionsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // MUTATION
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.mutation = new TMutation();
+ struct.mutation.read(iprot);
+ struct.setMutationIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.id = iprot.readI64();
+ struct.setIdIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot, TConditionalMutation struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.conditions != null) {
+ oprot.writeFieldBegin(CONDITIONS_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.conditions.size()));
+ for (TCondition _iter109 : struct.conditions)
+ {
+ _iter109.write(oprot);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.mutation != null) {
+ oprot.writeFieldBegin(MUTATION_FIELD_DESC);
+ struct.mutation.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(ID_FIELD_DESC);
+ oprot.writeI64(struct.id);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TConditionalMutationTupleSchemeFactory implements SchemeFactory {
+ public TConditionalMutationTupleScheme getScheme() {
+ return new TConditionalMutationTupleScheme();
+ }
+ }
+
+ private static class TConditionalMutationTupleScheme extends TupleScheme<TConditionalMutation> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TConditionalMutation struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetConditions()) {
+ optionals.set(0);
+ }
+ if (struct.isSetMutation()) {
+ optionals.set(1);
+ }
+ if (struct.isSetId()) {
+ optionals.set(2);
+ }
+ oprot.writeBitSet(optionals, 3);
+ if (struct.isSetConditions()) {
+ {
+ oprot.writeI32(struct.conditions.size());
+ for (TCondition _iter110 : struct.conditions)
+ {
+ _iter110.write(oprot);
+ }
+ }
+ }
+ if (struct.isSetMutation()) {
+ struct.mutation.write(oprot);
+ }
+ if (struct.isSetId()) {
+ oprot.writeI64(struct.id);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TConditionalMutation struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(3);
+ if (incoming.get(0)) {
+ {
+ org.apache.thrift.protocol.TList _list111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+ struct.conditions = new ArrayList<TCondition>(_list111.size);
+ for (int _i112 = 0; _i112 < _list111.size; ++_i112)
+ {
+ TCondition _elem113; // required
+ _elem113 = new TCondition();
+ _elem113.read(iprot);
+ struct.conditions.add(_elem113);
+ }
+ }
+ struct.setConditionsIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.mutation = new TMutation();
+ struct.mutation.read(iprot);
+ struct.setMutationIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.id = iprot.readI64();
+ struct.setIdIsSet(true);
+ }
+ }
+ }
+
+}
+
[6/6] git commit: ACCUMULO-1000 added conditional mutations to
Accumulo
Posted by kt...@apache.org.
ACCUMULO-1000 added conditional mutations to Accumulo
Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/9dc24448
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/9dc24448
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/9dc24448
Branch: refs/heads/master
Commit: 9dc244484b4d35859d4d22b27580a47ae7da0e1a
Parents: 77cac56
Author: Keith Turner <kt...@apache.org>
Authored: Wed Jul 24 15:48:48 2013 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Wed Jul 24 15:48:48 2013 -0400
----------------------------------------------------------------------
.../accumulo/core/client/ConditionalWriter.java | 138 +
.../core/client/ConditionalWriterConfig.java | 118 +
.../apache/accumulo/core/client/Connector.java | 15 +
.../core/client/impl/CompressedIterators.java | 131 +
.../core/client/impl/ConditionalWriterImpl.java | 794 ++++
.../core/client/impl/ConnectorImpl.java | 8 +-
.../core/client/impl/RootTabletLocator.java | 8 +-
.../accumulo/core/client/impl/ServerClient.java | 4 +-
.../core/client/impl/TabletLocator.java | 16 +-
.../core/client/impl/TabletLocatorImpl.java | 16 +-
.../client/impl/TabletServerBatchWriter.java | 34 +-
.../core/client/impl/TimeoutTabletLocator.java | 4 +-
.../core/client/mock/MockConnector.java | 8 +
.../core/client/mock/MockTabletLocator.java | 8 +-
.../accumulo/core/data/ArrayByteSequence.java | 13 +
.../apache/accumulo/core/data/Condition.java | 148 +
.../accumulo/core/data/ConditionalMutation.java | 83 +
.../org/apache/accumulo/core/data/Mutation.java | 130 +-
.../accumulo/core/data/thrift/TCMResult.java | 516 ++
.../accumulo/core/data/thrift/TCMStatus.java | 67 +
.../accumulo/core/data/thrift/TCondition.java | 1049 +++++
.../core/data/thrift/TConditionalMutation.java | 659 +++
.../core/data/thrift/TConditionalSession.java | 578 +++
.../accumulo/core/file/rfile/RelativeKey.java | 33 +-
.../iterators/system/ColumnQualifierFilter.java | 5 +-
.../thrift/TabletClientService.java | 4395 +++++++++++++++++-
.../accumulo/core/util/ByteBufferUtil.java | 13 +
.../core/util/UnsynchronizedBuffer.java | 195 +
core/src/main/thrift/data.thrift | 36 +
core/src/main/thrift/tabletserver.thrift | 10 +-
.../core/client/impl/TabletLocatorImplTest.java | 6 +-
.../core/file/rfile/RelativeKeyTest.java | 31 +-
.../server/data/ServerConditionalMutation.java | 58 +
.../server/security/SecurityOperation.java | 7 +
.../tabletserver/ConditionalMutationSet.java | 91 +
.../accumulo/server/tabletserver/RowLocks.java | 162 +
.../accumulo/server/tabletserver/Tablet.java | 8 +-
.../server/tabletserver/TabletServer.java | 308 +-
.../server/client/BulkImporterTest.java | 7 +-
.../accumulo/test/FaultyConditionalWriter.java | 81 +
.../accumulo/test/functional/BadIterator.java | 5 +
.../accumulo/test/functional/SlowIterator.java | 24 +-
.../test/performance/thrift/NullTserver.java | 23 +
.../accumulo/test/ConditionalWriterTest.java | 1236 +++++
44 files changed, 10840 insertions(+), 439 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java
new file mode 100644
index 0000000..4ed4d31
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java
@@ -0,0 +1,138 @@
+/*
+ * 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.accumulo.core.client;
+
+import java.util.Iterator;
+
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.data.ConditionalMutation;
+
+/**
+ * @since 1.6.0
+ */
+public interface ConditionalWriter {
+ public static class Result {
+
+ private Status status;
+ private ConditionalMutation mutation;
+ private String server;
+ private Exception exception;
+
+ public Result(Status s, ConditionalMutation m, String server) {
+ this.status = s;
+ this.mutation = m;
+ this.server = server;
+ }
+
+ public Result(Exception e, ConditionalMutation cm, String server) {
+ this.exception = e;
+ this.mutation = cm;
+ this.server = server;
+ }
+
+ /**
+ * If this method throws an exception, then its possible the mutation is still being actively processed. Therefore if code chooses to continue after seeing
+ * an exception it should take this into consideration.
+ *
+ * @return status of a conditional mutation
+ * @throws AccumuloException
+ * @throws AccumuloSecurityException
+ */
+
+ public Status getStatus() throws AccumuloException, AccumuloSecurityException {
+ if (status == null) {
+ if (exception instanceof AccumuloException)
+ throw new AccumuloException(exception);
+ if (exception instanceof AccumuloSecurityException) {
+ AccumuloSecurityException ase = (AccumuloSecurityException) exception;
+ throw new AccumuloSecurityException(ase.getUser(), SecurityErrorCode.valueOf(ase.getSecurityErrorCode().name()), ase.getTableInfo(), ase);
+ }
+ else
+ throw new AccumuloException(exception);
+ }
+
+ return status;
+ }
+
+ /**
+ *
+ * @return A copy of the mutation previously submitted by a user. The mutation will reference the same data, but the object may be different.
+ */
+ public ConditionalMutation getMutation() {
+ return mutation;
+ }
+
+ /**
+ *
+ * @return The server this mutation was sent to. Returns null if was not sent to a server.
+ */
+ public String getTabletServer() {
+ return server;
+ }
+ }
+
+ public static enum Status {
+ /**
+ * conditions were met and mutation was written
+ */
+ ACCEPTED,
+ /**
+ * conditions were not met and mutation was not written
+ */
+ REJECTED,
+ /**
+ * mutation violated a constraint and was not written
+ */
+ VIOLATED,
+ /**
+ * error occurred after mutation was sent to server, its unknown if the mutation was written. Although the status of the mutation is unknown, Accumulo
+ * guarantees the mutation will not be written at a later point in time.
+ */
+ UNKNOWN,
+ /**
+ * A condition contained a column visibility that could never be seen
+ */
+ INVISIBLE_VISIBILITY,
+
+ }
+
+ /**
+ * This method returns one result for each mutation passed to it. This method is thread safe. Multiple threads can safely use a single conditional writer.
+ * Sharing a conditional writer between multiple threads may result in batching of request to tablet servers.
+ *
+ * @param mutations
+ * @return Result for each mutation submitted. The mutations may still be processing in the background when this method returns, if so the iterator will
+ * block.
+ */
+ public abstract Iterator<Result> write(Iterator<ConditionalMutation> mutations);
+
+ /**
+ * This method has the same thread safety guarantees as @link {@link #write(Iterator)}
+ *
+ *
+ * @param mutation
+ * @return Result for the submitted mutation
+ */
+
+ public abstract Result write(ConditionalMutation mutation);
+
+ /**
+ * release any resources (like threads pools) used by conditional writer
+ */
+ public void close();
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
new file mode 100644
index 0000000..f2a91ea
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriterConfig.java
@@ -0,0 +1,118 @@
+/*
+ * 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.accumulo.core.client;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.ArgumentChecker;
+
+/**
+ *
+ * @since 1.6.0
+ */
+public class ConditionalWriterConfig {
+
+ private static final Long DEFAULT_TIMEOUT = Long.MAX_VALUE;
+ private Long timeout = null;
+
+ private static final Integer DEFAULT_MAX_WRITE_THREADS = 3;
+ private Integer maxWriteThreads = null;
+
+ private Authorizations auths = Authorizations.EMPTY;
+
+ /**
+ * A set of authorization labels that will be checked against the column visibility of each key in order to filter data. The authorizations passed in must be
+ * a subset of the accumulo user's set of authorizations. If the accumulo user has authorizations (A1, A2) and authorizations (A2, A3) are passed, then an
+ * exception will be thrown.
+ *
+ * <p>
+ * Any condition that is not visible with this set of authorizations will fail.
+ *
+ * @param auths
+ */
+ public ConditionalWriterConfig setAuthorizations(Authorizations auths) {
+ ArgumentChecker.notNull(auths);
+ this.auths = auths;
+ return this;
+ }
+
+ /**
+ * Sets the maximum amount of time an unresponsive server will be re-tried. When this timeout is exceeded, the {@link ConditionalWriter} should return the
+ * mutation with an exception.<br />
+ * For no timeout, set to zero, or {@link Long#MAX_VALUE} with {@link TimeUnit#MILLISECONDS}.
+ *
+ * <p>
+ * {@link TimeUnit#MICROSECONDS} or {@link TimeUnit#NANOSECONDS} will be truncated to the nearest {@link TimeUnit#MILLISECONDS}.<br />
+ * If this truncation would result in making the value zero when it was specified as non-zero, then a minimum value of one {@link TimeUnit#MILLISECONDS} will
+ * be used.
+ *
+ * <p>
+ * <b>Default:</b> {@link Long#MAX_VALUE} (no timeout)
+ *
+ * @param timeout
+ * the timeout, in the unit specified by the value of {@code timeUnit}
+ * @param timeUnit
+ * determines how {@code timeout} will be interpreted
+ * @throws IllegalArgumentException
+ * if {@code timeout} is less than 0
+ * @return {@code this} to allow chaining of set methods
+ */
+ public ConditionalWriterConfig setTimeout(long timeout, TimeUnit timeUnit) {
+ if (timeout < 0)
+ throw new IllegalArgumentException("Negative timeout not allowed " + timeout);
+
+ if (timeout == 0)
+ this.timeout = Long.MAX_VALUE;
+ else
+ // make small, positive values that truncate to 0 when converted use the minimum millis instead
+ this.timeout = Math.max(1, timeUnit.toMillis(timeout));
+ return this;
+ }
+
+ /**
+ * Sets the maximum number of threads to use for writing data to the tablet servers.
+ *
+ * <p>
+ * <b>Default:</b> 3
+ *
+ * @param maxWriteThreads
+ * the maximum threads to use
+ * @throws IllegalArgumentException
+ * if {@code maxWriteThreads} is non-positive
+ * @return {@code this} to allow chaining of set methods
+ */
+ public ConditionalWriterConfig setMaxWriteThreads(int maxWriteThreads) {
+ if (maxWriteThreads <= 0)
+ throw new IllegalArgumentException("Max threads must be positive " + maxWriteThreads);
+
+ this.maxWriteThreads = maxWriteThreads;
+ return this;
+ }
+
+ public Authorizations getAuthorizations() {
+ return auths;
+ }
+
+ public long getTimeout(TimeUnit timeUnit) {
+ return timeUnit.convert(timeout != null ? timeout : DEFAULT_TIMEOUT, TimeUnit.MILLISECONDS);
+ }
+
+ public int getMaxWriteThreads() {
+ return maxWriteThreads != null ? maxWriteThreads : DEFAULT_MAX_WRITE_THREADS;
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/Connector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index d2e7321..bbfa55f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -174,6 +174,21 @@ public abstract class Connector {
public abstract Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException;
/**
+ * Factory method to create a ConditionalWriter connected to Accumulo.
+ *
+ * @param tableName
+ * the name of the table to query data from
+ * @param config
+ * configuration used to create conditional writer
+ *
+ * @return ConditionalWriter object for writing ConditionalMutations
+ * @throws TableNotFoundException
+ * when the specified table doesn't exist
+ * @since 1.6.0
+ */
+ public abstract ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException;
+
+ /**
* Accessor method for internal instance object.
*
* @return the internal instance object
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java b/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java
new file mode 100644
index 0000000..549322e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/CompressedIterators.java
@@ -0,0 +1,131 @@
+/*
+ * 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.accumulo.core.client.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
+
+public class CompressedIterators {
+ private Map<String,Integer> symbolMap;
+ private List<String> symbolTable;
+ private Map<ByteSequence,IterConfig> cache;
+
+ public static class IterConfig {
+ public List<IterInfo> ssiList = new ArrayList<IterInfo>();
+ public Map<String,Map<String,String>> ssio = new HashMap<String,Map<String,String>>();
+ }
+
+ public CompressedIterators() {
+ symbolMap = new HashMap<String,Integer>();
+ symbolTable = new ArrayList<String>();
+ }
+
+ public CompressedIterators(List<String> symbols) {
+ this.symbolTable = symbols;
+ this.cache = new HashMap<ByteSequence,IterConfig>();
+ }
+
+ private int getSymbolID(String symbol) {
+ Integer id = symbolMap.get(symbol);
+ if (id == null) {
+ id = symbolTable.size();
+ symbolTable.add(symbol);
+ symbolMap.put(symbol, id);
+ }
+
+ return id;
+ }
+
+ public ByteBuffer compress(IteratorSetting[] iterators) {
+
+ UnsynchronizedBuffer.Writer out = new UnsynchronizedBuffer.Writer(iterators.length * 8);
+
+ out.writeVInt(iterators.length);
+
+ for (IteratorSetting is : iterators) {
+ out.writeVInt(getSymbolID(is.getName()));
+ out.writeVInt(getSymbolID(is.getIteratorClass()));
+ out.writeVInt(is.getPriority());
+
+ Map<String,String> opts = is.getOptions();
+ out.writeVInt(opts.size());
+
+ for (Entry<String,String> entry : opts.entrySet()) {
+ out.writeVInt(getSymbolID(entry.getKey()));
+ out.writeVInt(getSymbolID(entry.getValue()));
+ }
+ }
+
+ return out.toByteBuffer();
+
+ }
+
+ public IterConfig decompress(ByteBuffer iterators) {
+
+ ByteSequence iterKey = new ArrayByteSequence(iterators);
+ IterConfig config = cache.get(iterKey);
+ if (config != null) {
+ return config;
+ }
+
+ config = new IterConfig();
+
+ UnsynchronizedBuffer.Reader in = new UnsynchronizedBuffer.Reader(iterators);
+
+ int num = in.readVInt();
+
+ for (int i = 0; i < num; i++) {
+ String name = symbolTable.get(in.readVInt());
+ String iterClass = symbolTable.get(in.readVInt());
+ int prio = in.readVInt();
+
+ config.ssiList.add(new IterInfo(prio, iterClass, name));
+
+ int numOpts = in.readVInt();
+
+ HashMap<String,String> opts = new HashMap<String,String>();
+
+ for (int j = 0; j < numOpts; j++) {
+ String key = symbolTable.get(in.readVInt());
+ String val = symbolTable.get(in.readVInt());
+
+ opts.put(key, val);
+ }
+
+ config.ssio.put(name, opts);
+
+ }
+
+ cache.put(iterKey, config);
+ return config;
+ }
+
+ public List<String> getSymbolTable() {
+ return symbolTable;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
new file mode 100644
index 0000000..9d7e257
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -0,0 +1,794 @@
+/*
+ * 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.accumulo.core.client.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriterConfig;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.TimedOutException;
+import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Condition;
+import org.apache.accumulo.core.data.ConditionalMutation;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.thrift.TCMResult;
+import org.apache.accumulo.core.data.thrift.TCMStatus;
+import org.apache.accumulo.core.data.thrift.TCondition;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+import org.apache.accumulo.core.data.thrift.TConditionalSession;
+import org.apache.accumulo.core.data.thrift.TKeyExtent;
+import org.apache.accumulo.core.data.thrift.TMutation;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.VisibilityEvaluator;
+import org.apache.accumulo.core.security.VisibilityParseException;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.util.BadArgumentException;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.LoggingRunnable;
+import org.apache.accumulo.core.util.ThriftUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooLock;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.LockID;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.accumulo.trace.thrift.TInfo;
+import org.apache.commons.collections.map.LRUMap;
+import org.apache.commons.lang.mutable.MutableLong;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.transport.TTransportException;
+
+
+class ConditionalWriterImpl implements ConditionalWriter {
+
+ private static ThreadPoolExecutor cleanupThreadPool = new ThreadPoolExecutor(1, 1, 10, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
+
+ static {
+ cleanupThreadPool.allowCoreThreadTimeOut(true);
+ }
+
+ private static final Logger log = Logger.getLogger(ConditionalWriterImpl.class);
+
+ private static final int MAX_SLEEP = 30000;
+
+ private Authorizations auths;
+ private VisibilityEvaluator ve;
+ @SuppressWarnings("unchecked")
+ private Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap(1000));
+ private Instance instance;
+ private TCredentials credentials;
+ private TabletLocator locator;
+ private String tableId;
+ private long timeout;
+
+ private static class ServerQueue {
+ BlockingQueue<TabletServerMutations<QCMutation>> queue = new LinkedBlockingQueue<TabletServerMutations<QCMutation>>();
+ boolean taskQueued = false;
+ }
+
+ private Map<String,ServerQueue> serverQueues;
+ private DelayQueue<QCMutation> failedMutations = new DelayQueue<QCMutation>();
+ private ScheduledThreadPoolExecutor threadPool;
+
+ private class RQIterator implements Iterator<Result> {
+
+ private BlockingQueue<Result> rq;
+ private int count;
+
+ public RQIterator(BlockingQueue<Result> resultQueue, int count) {
+ this.rq = resultQueue;
+ this.count = count;
+ }
+
+ @Override
+ public boolean hasNext() {
+ return count > 0;
+ }
+
+ @Override
+ public Result next() {
+ if (count <= 0)
+ throw new NoSuchElementException();
+
+ try {
+ Result result = rq.poll(1, TimeUnit.SECONDS);
+ while (result == null) {
+
+ if (threadPool.isShutdown()) {
+ throw new NoSuchElementException("ConditionalWriter closed");
+ }
+
+ result = rq.poll(1, TimeUnit.SECONDS);
+ }
+ count--;
+ return result;
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+
+ }
+
+ private static class QCMutation extends ConditionalMutation implements Delayed {
+ private BlockingQueue<Result> resultQueue;
+ private long resetTime;
+ private long delay = 50;
+ private long entryTime;
+
+ QCMutation(ConditionalMutation cm, BlockingQueue<Result> resultQueue, long entryTime) {
+ super(cm);
+ this.resultQueue = resultQueue;
+ this.entryTime = entryTime;
+ }
+
+ @Override
+ public int compareTo(Delayed o) {
+ QCMutation oqcm = (QCMutation) o;
+ return Long.valueOf(resetTime).compareTo(Long.valueOf(oqcm.resetTime));
+ }
+
+ @Override
+ public long getDelay(TimeUnit unit) {
+ return unit.convert(delay - (System.currentTimeMillis() - resetTime), TimeUnit.MILLISECONDS);
+ }
+
+ void resetDelay() {
+ delay = Math.min(delay * 2, MAX_SLEEP);
+ resetTime = System.currentTimeMillis();
+ }
+
+ void queueResult(Result result) {
+ resultQueue.add(result);
+ }
+ }
+
+ private ServerQueue getServerQueue(String location) {
+ ServerQueue serverQueue;
+ synchronized (serverQueues) {
+ serverQueue = serverQueues.get(location);
+ if (serverQueue == null) {
+
+ serverQueue = new ServerQueue();
+ serverQueues.put(location, serverQueue);
+ }
+ }
+ return serverQueue;
+ }
+
+ private class CleanupTask implements Runnable {
+ private List<SessionID> sessions;
+
+ CleanupTask(List<SessionID> activeSessions) {
+ this.sessions = activeSessions;
+ }
+
+ @Override
+ public void run() {
+ TabletClientService.Iface client = null;
+
+ for (SessionID sid : sessions) {
+ if (!sid.isActive())
+ continue;
+
+ TInfo tinfo = Tracer.traceInfo();
+ try {
+ client = getClient(sid.location);
+ client.closeConditionalUpdate(tinfo, sid.sessionID);
+ } catch (Exception e) {
+ } finally {
+ ThriftUtil.returnClient((TServiceClient) client);
+ }
+
+ }
+ }
+ }
+
+ private void queueRetry(List<QCMutation> mutations, String server) {
+
+ if (timeout < Long.MAX_VALUE) {
+
+ long time = System.currentTimeMillis();
+
+ ArrayList<QCMutation> mutations2 = new ArrayList<ConditionalWriterImpl.QCMutation>(mutations.size());
+
+ for (QCMutation qcm : mutations) {
+ qcm.resetDelay();
+ if (time + qcm.getDelay(TimeUnit.MILLISECONDS) > qcm.entryTime + timeout) {
+ TimedOutException toe;
+ if (server != null)
+ toe = new TimedOutException(Collections.singleton(server));
+ else
+ toe = new TimedOutException("Conditional mutation timed out");
+
+ qcm.queueResult(new Result(toe, qcm, server));
+ } else {
+ mutations2.add(qcm);
+ }
+ }
+
+ if (mutations2.size() > 0)
+ failedMutations.addAll(mutations2);
+
+ } else {
+ for (QCMutation qcm : mutations)
+ qcm.resetDelay();
+ failedMutations.addAll(mutations);
+ }
+ }
+
+ private void queue(List<QCMutation> mutations) {
+ List<QCMutation> failures = new ArrayList<QCMutation>();
+ Map<String,TabletServerMutations<QCMutation>> binnedMutations = new HashMap<String,TabletLocator.TabletServerMutations<QCMutation>>();
+
+ try {
+ locator.binMutations(mutations, binnedMutations, failures, credentials);
+
+ if (failures.size() == mutations.size())
+ if (!Tables.exists(instance, tableId))
+ throw new TableDeletedException(tableId);
+ else if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+ throw new TableOfflineException(instance, tableId);
+
+ } catch (Exception e) {
+ for (QCMutation qcm : mutations)
+ qcm.queueResult(new Result(e, qcm, null));
+
+ // do not want to queue anything that was put in before binMutations() failed
+ failures.clear();
+ binnedMutations.clear();
+ }
+
+ if (failures.size() > 0)
+ queueRetry(failures, null);
+
+ for (Entry<String,TabletServerMutations<QCMutation>> entry : binnedMutations.entrySet()) {
+ queue(entry.getKey(), entry.getValue());
+ }
+
+
+ }
+
+ private void queue(String location, TabletServerMutations<QCMutation> mutations) {
+
+ ServerQueue serverQueue = getServerQueue(location);
+
+ synchronized (serverQueue) {
+ serverQueue.queue.add(mutations);
+ // never execute more than one task per server
+ if(!serverQueue.taskQueued){
+ threadPool.execute(new LoggingRunnable(log, new SendTask(location)));
+ serverQueue.taskQueued = true;
+ }
+ }
+
+ }
+
+ private void reschedule(SendTask task){
+ ServerQueue serverQueue = getServerQueue(task.location);
+ // just finished processing work for this server, could reschedule if it has more work or immediately process the work
+ // this code reschedules the the server for processing later... there may be other queues with
+ // more data that need to be processed... also it will give the current server time to build
+ // up more data... the thinking is that rescheduling instead or processing immediately will result
+ // in bigger batches and less RPC overhead
+
+ synchronized (serverQueue) {
+ if(serverQueue.queue.size() > 0)
+ threadPool.execute(new LoggingRunnable(log, task));
+ else
+ serverQueue.taskQueued = false;
+ }
+
+ }
+
+ private TabletServerMutations<QCMutation> dequeue(String location) {
+ BlockingQueue<TabletServerMutations<QCMutation>> queue = getServerQueue(location).queue;
+
+ ArrayList<TabletServerMutations<QCMutation>> mutations = new ArrayList<TabletLocator.TabletServerMutations<QCMutation>>();
+ queue.drainTo(mutations);
+
+ if (mutations.size() == 0)
+ return null;
+
+ if (mutations.size() == 1) {
+ return mutations.get(0);
+ } else {
+ // merge multiple request to a single tablet server
+ TabletServerMutations<QCMutation> tsm = mutations.get(0);
+
+ for (int i = 1; i < mutations.size(); i++) {
+ for (Entry<KeyExtent,List<QCMutation>> entry : mutations.get(i).getMutations().entrySet()) {
+ List<QCMutation> list = tsm.getMutations().get(entry.getKey());
+ if (list == null) {
+ list = new ArrayList<QCMutation>();
+ tsm.getMutations().put(entry.getKey(), list);
+ }
+
+ list.addAll(entry.getValue());
+ }
+ }
+
+ return tsm;
+ }
+ }
+
+ ConditionalWriterImpl(Instance instance, TCredentials credentials, String tableId, ConditionalWriterConfig config) {
+ this.instance = instance;
+ this.credentials = credentials;
+ this.auths = config.getAuthorizations();
+ this.ve = new VisibilityEvaluator(config.getAuthorizations());
+ this.threadPool = new ScheduledThreadPoolExecutor(config.getMaxWriteThreads());
+ this.threadPool.setMaximumPoolSize(config.getMaxWriteThreads());
+ this.locator = TabletLocator.getLocator(instance, new Text(tableId));
+ this.serverQueues = new HashMap<String,ServerQueue>();
+ this.tableId = tableId;
+ this.timeout = config.getTimeout(TimeUnit.MILLISECONDS);
+
+ Runnable failureHandler = new Runnable() {
+
+ @Override
+ public void run() {
+ List<QCMutation> mutations = new ArrayList<QCMutation>();
+ failedMutations.drainTo(mutations);
+ queue(mutations);
+ }
+ };
+
+ failureHandler = new LoggingRunnable(log, failureHandler);
+
+ threadPool.scheduleAtFixedRate(failureHandler, 250, 250, TimeUnit.MILLISECONDS);
+ }
+
+ public Iterator<Result> write(Iterator<ConditionalMutation> mutations) {
+
+ BlockingQueue<Result> resultQueue = new LinkedBlockingQueue<Result>();
+
+ List<QCMutation> mutationList = new ArrayList<QCMutation>();
+
+ int count = 0;
+
+ long entryTime = System.currentTimeMillis();
+
+ mloop: while (mutations.hasNext()) {
+ ConditionalMutation mut = mutations.next();
+ count++;
+
+ for (Condition cond : mut.getConditions()) {
+ if (!isVisible(cond.getVisibility())) {
+ resultQueue.add(new Result(Status.INVISIBLE_VISIBILITY, mut, null));
+ continue mloop;
+ }
+ }
+
+ // copy the mutations so that even if caller changes it, it will not matter
+ mutationList.add(new QCMutation(mut, resultQueue, entryTime));
+ }
+
+ queue(mutationList);
+
+ return new RQIterator(resultQueue, count);
+
+ }
+
+ private class SendTask implements Runnable {
+
+
+ String location;
+
+ public SendTask(String location) {
+ this.location = location;
+
+ }
+
+ @Override
+ public void run() {
+ try {
+ TabletServerMutations<QCMutation> mutations = dequeue(location);
+ if (mutations != null)
+ sendToServer(location, mutations);
+ } finally {
+ reschedule(this);
+ }
+ }
+ }
+
+ private static class CMK {
+
+ QCMutation cm;
+ KeyExtent ke;
+
+ public CMK(KeyExtent ke, QCMutation cm) {
+ this.ke = ke;
+ this.cm = cm;
+ }
+ }
+
+ private static class SessionID {
+ String location;
+ String lockId;
+ long sessionID;
+ boolean reserved;
+ long lastAccessTime;
+ long ttl;
+
+ boolean isActive() {
+ return System.currentTimeMillis() - lastAccessTime < ttl * .95;
+ }
+ }
+
+ private HashMap<String, SessionID> cachedSessionIDs = new HashMap<String, SessionID>();
+
+ private SessionID reserveSessionID(String location, TabletClientService.Iface client, TInfo tinfo) throws ThriftSecurityException, TException {
+ //avoid cost of repeatedly making RPC to create sessions, reuse sessions
+ synchronized (cachedSessionIDs) {
+ SessionID sid = cachedSessionIDs.get(location);
+ if (sid != null) {
+ if (sid.reserved)
+ throw new IllegalStateException();
+
+ if (!sid.isActive()) {
+ cachedSessionIDs.remove(location);
+ } else {
+ sid.reserved = true;
+ return sid;
+ }
+ }
+ }
+
+ TConditionalSession tcs = client.startConditionalUpdate(tinfo, credentials, ByteBufferUtil.toByteBuffers(auths.getAuthorizations()), tableId);
+
+ synchronized (cachedSessionIDs) {
+ SessionID sid = new SessionID();
+ sid.reserved = true;
+ sid.sessionID = tcs.sessionId;
+ sid.lockId = tcs.tserverLock;
+ sid.ttl = tcs.ttl;
+ sid.location = location;
+ if (cachedSessionIDs.put(location, sid) != null)
+ throw new IllegalStateException();
+
+ return sid;
+ }
+
+ }
+
+ private void invalidateSessionID(String location) {
+ synchronized (cachedSessionIDs) {
+ cachedSessionIDs.remove(location);
+ }
+
+ }
+
+ private void unreserveSessionID(String location){
+ synchronized (cachedSessionIDs) {
+ SessionID sid = cachedSessionIDs.get(location);
+ if (sid != null) {
+ if (!sid.reserved)
+ throw new IllegalStateException();
+ sid.reserved = false;
+ sid.lastAccessTime = System.currentTimeMillis();
+ }
+ }
+ }
+
+ List<SessionID> getActiveSessions() {
+ ArrayList<SessionID> activeSessions = new ArrayList<SessionID>();
+ for (SessionID sid : cachedSessionIDs.values())
+ if (sid.isActive())
+ activeSessions.add(sid);
+ return activeSessions;
+ }
+
+ private TabletClientService.Iface getClient(String location) throws TTransportException {
+ TabletClientService.Iface client;
+ if (timeout < instance.getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
+ client = ThriftUtil.getTServerClient(location, timeout);
+ else
+ client = ThriftUtil.getTServerClient(location, instance.getConfiguration());
+ return client;
+ }
+
+ private void sendToServer(String location, TabletServerMutations<QCMutation> mutations) {
+ TabletClientService.Iface client = null;
+
+ TInfo tinfo = Tracer.traceInfo();
+
+ Map<Long,CMK> cmidToCm = new HashMap<Long,CMK>();
+ MutableLong cmid = new MutableLong(0);
+
+ SessionID sessionId = null;
+
+ try {
+
+ client = getClient(location);
+
+ Map<TKeyExtent,List<TConditionalMutation>> tmutations = new HashMap<TKeyExtent,List<TConditionalMutation>>();
+
+ CompressedIterators compressedIters = new CompressedIterators();
+ convertMutations(mutations, cmidToCm, cmid, tmutations, compressedIters);
+
+ List<TCMResult> tresults = null;
+ while (tresults == null) {
+ try {
+ sessionId = reserveSessionID(location, client, tinfo);
+ tresults = client.conditionalUpdate(tinfo, sessionId.sessionID, tmutations, compressedIters.getSymbolTable());
+ } catch (NoSuchScanIDException nssie) {
+ sessionId = null;
+ invalidateSessionID(location);
+ }
+ }
+
+ HashSet<KeyExtent> extentsToInvalidate = new HashSet<KeyExtent>();
+
+ ArrayList<QCMutation> ignored = new ArrayList<QCMutation>();
+
+ for (TCMResult tcmResult : tresults) {
+ if (tcmResult.status == TCMStatus.IGNORED) {
+ CMK cmk = cmidToCm.get(tcmResult.cmid);
+ ignored.add(cmk.cm);
+ extentsToInvalidate.add(cmk.ke);
+ } else {
+ QCMutation qcm = cmidToCm.get(tcmResult.cmid).cm;
+ qcm.queueResult(new Result(fromThrift(tcmResult.status), qcm, location));
+ }
+ }
+
+ for (KeyExtent ke : extentsToInvalidate) {
+ locator.invalidateCache(ke);
+ }
+
+ queueRetry(ignored, location);
+
+ } catch (ThriftSecurityException tse) {
+ AccumuloSecurityException ase = new AccumuloSecurityException(credentials.getPrincipal(), tse.getCode(), Tables.getPrintableTableInfoFromId(instance,
+ tableId), tse);
+ queueException(location, cmidToCm, ase);
+ } catch (TTransportException e) {
+ locator.invalidateCache(location);
+ invalidateSession(location, mutations, cmidToCm, sessionId);
+ } catch (TApplicationException tae) {
+ queueException(location, cmidToCm, new AccumuloServerException(location, tae));
+ } catch (TException e) {
+ locator.invalidateCache(location);
+ invalidateSession(location, mutations, cmidToCm, sessionId);
+ } catch (Exception e) {
+ queueException(location, cmidToCm, e);
+ } finally {
+ unreserveSessionID(location);
+ ThriftUtil.returnClient((TServiceClient) client);
+ }
+ }
+
+
+ private void queueRetry(Map<Long,CMK> cmidToCm, String location) {
+ ArrayList<QCMutation> ignored = new ArrayList<QCMutation>();
+ for (CMK cmk : cmidToCm.values())
+ ignored.add(cmk.cm);
+ queueRetry(ignored, location);
+ }
+
+ private void queueException(String location, Map<Long,CMK> cmidToCm, Exception e) {
+ for (CMK cmk : cmidToCm.values())
+ cmk.cm.queueResult(new Result(e, cmk.cm, location));
+ }
+
+ private void invalidateSession(String location, TabletServerMutations<QCMutation> mutations, Map<Long,CMK> cmidToCm, SessionID sessionId) {
+ if(sessionId == null){
+ queueRetry(cmidToCm, location);
+ }else{
+ try {
+ invalidateSession(sessionId, location, mutations);
+ for (CMK cmk : cmidToCm.values())
+ cmk.cm.queueResult(new Result(Status.UNKNOWN, cmk.cm, location));
+ }catch(Exception e2){
+ queueException(location, cmidToCm, e2);
+ }
+ }
+ }
+
+ /*
+ * The purpose of this code is to ensure that a conditional mutation will not execute when its status is unknown. This allows a user to read the row when the
+ * status is unknown and not have to worry about the tserver applying the mutation after the scan.
+ *
+ * If a conditional mutation is taking a long time to process, then this method will wait for it to finish... unless this exceeds timeout.
+ */
+ private void invalidateSession(SessionID sessionId, String location, TabletServerMutations<QCMutation> mutations) throws AccumuloException,
+ AccumuloSecurityException, TableNotFoundException {
+
+ ArrayList<QCMutation> mutList = new ArrayList<QCMutation>();
+
+ for (List<QCMutation> tml : mutations.getMutations().values()) {
+ mutList.addAll(tml);
+ }
+
+ long sleepTime = 50;
+
+ long startTime = System.currentTimeMillis();
+
+ LockID lid = new LockID(ZooUtil.getRoot(instance) + Constants.ZTSERVERS, sessionId.lockId);
+
+ while (true) {
+ if (!ZooLock.isLockHeld(ServerClient.getZooCache(instance), lid)) {
+ // TODO if ACCUMULO-1152 adds a tserver lock check to the tablet location cache, then this invalidation would prevent future attempts to contact the
+ // tserver even its gone zombie and is still running w/o a lock
+ locator.invalidateCache(location);
+ return;
+ }
+
+ try {
+ // if the mutation is currently processing, this method will block until its done or times out
+ invalidateSession(sessionId.sessionID, location);
+
+ return;
+ } catch (TApplicationException tae) {
+ throw new AccumuloServerException(location, tae);
+ } catch (TException e) {
+ locator.invalidateCache(location);
+ }
+
+ if ((System.currentTimeMillis() - startTime) + sleepTime > timeout)
+ throw new TimedOutException(Collections.singleton(location));
+
+ UtilWaitThread.sleep(sleepTime);
+ sleepTime = Math.min(2 * sleepTime, MAX_SLEEP);
+
+ }
+
+ }
+
+ private void invalidateSession(long sessionId, String location) throws TException {
+ TabletClientService.Iface client = null;
+
+ TInfo tinfo = Tracer.traceInfo();
+
+ try {
+ client = getClient(location);
+ client.invalidateConditionalUpdate(tinfo, sessionId);
+ } finally {
+ ThriftUtil.returnClient((TServiceClient) client);
+ }
+ }
+
+ private Status fromThrift(TCMStatus status) {
+ switch (status) {
+ case ACCEPTED:
+ return Status.ACCEPTED;
+ case REJECTED:
+ return Status.REJECTED;
+ case VIOLATED:
+ return Status.VIOLATED;
+ default:
+ throw new IllegalArgumentException(status.toString());
+ }
+ }
+
+ private void convertMutations(TabletServerMutations<QCMutation> mutations, Map<Long,CMK> cmidToCm, MutableLong cmid,
+ Map<TKeyExtent,List<TConditionalMutation>> tmutations, CompressedIterators compressedIters) {
+
+ for (Entry<KeyExtent,List<QCMutation>> entry : mutations.getMutations().entrySet()) {
+ TKeyExtent tke = entry.getKey().toThrift();
+ ArrayList<TConditionalMutation> tcondMutaions = new ArrayList<TConditionalMutation>();
+
+ List<QCMutation> condMutations = entry.getValue();
+
+ for (QCMutation cm : condMutations) {
+ TMutation tm = cm.toThrift();
+
+ List<TCondition> conditions = convertConditions(cm, compressedIters);
+
+ cmidToCm.put(cmid.longValue(), new CMK(entry.getKey(), cm));
+ TConditionalMutation tcm = new TConditionalMutation(conditions, tm, cmid.longValue());
+ cmid.increment();
+ tcondMutaions.add(tcm);
+ }
+
+ tmutations.put(tke, tcondMutaions);
+ }
+ }
+
+ private List<TCondition> convertConditions(ConditionalMutation cm, CompressedIterators compressedIters) {
+ List<TCondition> conditions = new ArrayList<TCondition>(cm.getConditions().size());
+
+ for (Condition cond : cm.getConditions()) {
+ long ts = 0;
+ boolean hasTs = false;
+
+ if (cond.getTimestamp() != null) {
+ ts = cond.getTimestamp();
+ hasTs = true;
+ }
+
+ ByteBuffer iters = compressedIters.compress(cond.getIterators());
+
+ TCondition tc = new TCondition(ByteBufferUtil.toByteBuffers(cond.getFamily()), ByteBufferUtil.toByteBuffers(cond.getQualifier()),
+ ByteBufferUtil.toByteBuffers(cond.getVisibility()), ts, hasTs, ByteBufferUtil.toByteBuffers(cond.getValue()), iters);
+
+ conditions.add(tc);
+ }
+
+ return conditions;
+ }
+
+ private boolean isVisible(ByteSequence cv) {
+ Text testVis = new Text(cv.toArray());
+ if (testVis.getLength() == 0)
+ return true;
+
+ Boolean b = cache.get(testVis);
+ if (b != null)
+ return b;
+
+ try {
+ Boolean bb = ve.evaluate(new ColumnVisibility(testVis));
+ cache.put(new Text(testVis), bb);
+ return bb;
+ } catch (VisibilityParseException e) {
+ return false;
+ } catch (BadArgumentException e) {
+ return false;
+ }
+ }
+
+ public Result write(ConditionalMutation mutation) {
+ return write(Collections.singleton(mutation).iterator()).next();
+ }
+
+ @Override
+ public void close() {
+ threadPool.shutdownNow();
+ cleanupThreadPool.execute(new CleanupTask(getActiveSessions()));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index 3858cdc..57e36fd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -24,6 +24,8 @@ import org.apache.accumulo.core.client.BatchDeleter;
import org.apache.accumulo.core.client.BatchScanner;
import org.apache.accumulo.core.client.BatchWriter;
import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriterConfig;
import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.Instance;
import org.apache.accumulo.core.client.MultiTableBatchWriter;
@@ -130,6 +132,11 @@ public class ConnectorImpl extends Connector {
}
@Override
+ public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException {
+ return new ConditionalWriterImpl(instance, credentials, getTableId(tableName), config);
+ }
+
+ @Override
public Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException {
ArgumentChecker.notNull(tableName, authorizations);
return new ScannerImpl(instance, credentials, getTableId(tableName), authorizations);
@@ -162,5 +169,4 @@ public class ConnectorImpl extends Connector {
return instanceops;
}
-
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index 18b2a27..88e5c3a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -42,12 +42,12 @@ public class RootTabletLocator extends TabletLocator {
}
@Override
- public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+ public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+ TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
String rootTabletLocation = instance.getRootTabletLocation();
if (rootTabletLocation != null) {
- TabletServerMutations tsm = new TabletServerMutations();
- for (Mutation mutation : mutations) {
+ TabletServerMutations<T> tsm = new TabletServerMutations<T>();
+ for (T mutation : mutations) {
tsm.addMutation(RootTable.EXTENT, mutation);
}
binnedMutations.put(rootTabletLocation, tsm);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index b933c2b..218bd36 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -25,8 +25,8 @@ import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.Instance;
import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.util.ArgumentChecker;
@@ -45,7 +45,7 @@ public class ServerClient {
private static final Logger log = Logger.getLogger(ServerClient.class);
private static final Map<String,ZooCache> zooCaches = new HashMap<String,ZooCache>();
- private synchronized static ZooCache getZooCache(Instance instance) {
+ synchronized static ZooCache getZooCache(Instance instance) {
ZooCache result = zooCaches.get(instance.getZooKeepers());
if (result == null) {
result = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), null);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
index de8e053..f9110b2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
@@ -43,7 +43,7 @@ public abstract class TabletLocator {
public abstract TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException,
AccumuloSecurityException, TableNotFoundException;
- public abstract void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures,
+ public abstract <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
public abstract List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials)
@@ -187,24 +187,24 @@ public abstract class TabletLocator {
}
}
- public static class TabletServerMutations {
- private Map<KeyExtent,List<Mutation>> mutations;
+ public static class TabletServerMutations<T extends Mutation> {
+ private Map<KeyExtent,List<T>> mutations;
public TabletServerMutations() {
- mutations = new HashMap<KeyExtent,List<Mutation>>();
+ mutations = new HashMap<KeyExtent,List<T>>();
}
- public void addMutation(KeyExtent ke, Mutation m) {
- List<Mutation> mutList = mutations.get(ke);
+ public void addMutation(KeyExtent ke, T m) {
+ List<T> mutList = mutations.get(ke);
if (mutList == null) {
- mutList = new ArrayList<Mutation>();
+ mutList = new ArrayList<T>();
mutations.put(ke, mutList);
}
mutList.add(m);
}
- public Map<KeyExtent,List<Mutation>> getMutations() {
+ public Map<KeyExtent,List<T>> getMutations() {
return mutations;
}
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
index df5d66b..4b2e1d3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
@@ -113,14 +113,14 @@ public class TabletLocatorImpl extends TabletLocator {
}
@Override
- public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+ public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+ TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
OpTimer opTimer = null;
if (log.isTraceEnabled())
opTimer = new OpTimer(log, Level.TRACE).start("Binning " + mutations.size() + " mutations for table " + tableId);
- ArrayList<Mutation> notInCache = new ArrayList<Mutation>();
+ ArrayList<T> notInCache = new ArrayList<T>();
Text row = new Text();
rLock.lock();
@@ -133,7 +133,7 @@ public class TabletLocatorImpl extends TabletLocator {
// For this to be efficient, need to avoid fine grained synchronization and fine grained logging.
// Therefore methods called by this are not synchronized and should not log.
- for (Mutation mutation : mutations) {
+ for (T mutation : mutations) {
row.set(mutation.getRow());
TabletLocation tl = locateTabletInCache(row);
if (tl == null)
@@ -157,7 +157,7 @@ public class TabletLocatorImpl extends TabletLocator {
wLock.lock();
try {
boolean failed = false;
- for (Mutation mutation : notInCache) {
+ for (T mutation : notInCache) {
if (failed) {
// when one table does not return a location, something is probably
// screwy, go ahead and fail everything.
@@ -185,11 +185,11 @@ public class TabletLocatorImpl extends TabletLocator {
opTimer.stop("Binned " + mutations.size() + " mutations for table " + tableId + " to " + binnedMutations.size() + " tservers in %DURATION%");
}
- private void addMutation(Map<String,TabletServerMutations> binnedMutations, Mutation mutation, TabletLocation tl) {
- TabletServerMutations tsm = binnedMutations.get(tl.tablet_location);
+ private <T extends Mutation> void addMutation(Map<String,TabletServerMutations<T>> binnedMutations, T mutation, TabletLocation tl) {
+ TabletServerMutations<T> tsm = binnedMutations.get(tl.tablet_location);
if (tsm == null) {
- tsm = new TabletServerMutations();
+ tsm = new TabletServerMutations<T>();
binnedMutations.put(tl.tablet_location, tsm);
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index 766cea9..12f5243 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -415,13 +415,13 @@ public class TabletServerBatchWriter {
totalSendTime.addAndGet(time);
}
- public void updateBinningStats(int count, long time, Map<String,TabletServerMutations> binnedMutations) {
+ public void updateBinningStats(int count, long time, Map<String,TabletServerMutations<Mutation>> binnedMutations) {
totalBinTime.addAndGet(time);
totalBinned.addAndGet(count);
updateBatchStats(binnedMutations);
}
- private synchronized void updateBatchStats(Map<String,TabletServerMutations> binnedMutations) {
+ private synchronized void updateBatchStats(Map<String,TabletServerMutations<Mutation>> binnedMutations) {
tabletServersBatchSum += binnedMutations.size();
minTabletServersBatch = Math.min(minTabletServersBatch, binnedMutations.size());
@@ -429,8 +429,8 @@ public class TabletServerBatchWriter {
int numTablets = 0;
- for (Entry<String,TabletServerMutations> entry : binnedMutations.entrySet()) {
- TabletServerMutations tsm = entry.getValue();
+ for (Entry<String,TabletServerMutations<Mutation>> entry : binnedMutations.entrySet()) {
+ TabletServerMutations<Mutation> tsm = entry.getValue();
numTablets += tsm.getMutations().size();
}
@@ -577,7 +577,7 @@ public class TabletServerBatchWriter {
init().addAll(failures);
}
- synchronized void add(String location, TabletServerMutations tsm) {
+ synchronized void add(String location, TabletServerMutations<Mutation> tsm) {
init();
for (Entry<KeyExtent,List<Mutation>> entry : tsm.getMutations().entrySet()) {
recentFailures.addAll(entry.getKey().getTableId().toString(), entry.getValue());
@@ -617,12 +617,12 @@ public class TabletServerBatchWriter {
private static final int MUTATION_BATCH_SIZE = 1 << 17;
private ExecutorService sendThreadPool;
- private Map<String,TabletServerMutations> serversMutations;
+ private Map<String,TabletServerMutations<Mutation>> serversMutations;
private Set<String> queued;
private Map<String,TabletLocator> locators;
public MutationWriter(int numSendThreads) {
- serversMutations = new HashMap<String,TabletServerMutations>();
+ serversMutations = new HashMap<String,TabletServerMutations<Mutation>>();
queued = new HashSet<String>();
sendThreadPool = new SimpleThreadPool(numSendThreads, this.getClass().getName());
locators = new HashMap<String,TabletLocator>();
@@ -639,7 +639,7 @@ public class TabletServerBatchWriter {
return ret;
}
- private void binMutations(MutationSet mutationsToProcess, Map<String,TabletServerMutations> binnedMutations) {
+ private void binMutations(MutationSet mutationsToProcess, Map<String,TabletServerMutations<Mutation>> binnedMutations) {
try {
Set<Entry<String,List<Mutation>>> es = mutationsToProcess.getMutations().entrySet();
for (Entry<String,List<Mutation>> entry : es) {
@@ -687,7 +687,7 @@ public class TabletServerBatchWriter {
}
void addMutations(MutationSet mutationsToSend) {
- Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
+ Map<String,TabletServerMutations<Mutation>> binnedMutations = new HashMap<String,TabletServerMutations<Mutation>>();
Span span = Trace.start("binMutations");
try {
long t1 = System.currentTimeMillis();
@@ -700,15 +700,15 @@ public class TabletServerBatchWriter {
addMutations(binnedMutations);
}
- private synchronized void addMutations(Map<String,TabletServerMutations> binnedMutations) {
+ private synchronized void addMutations(Map<String,TabletServerMutations<Mutation>> binnedMutations) {
int count = 0;
// merge mutations into existing mutations for a tablet server
- for (Entry<String,TabletServerMutations> entry : binnedMutations.entrySet()) {
+ for (Entry<String,TabletServerMutations<Mutation>> entry : binnedMutations.entrySet()) {
String server = entry.getKey();
- TabletServerMutations currentMutations = serversMutations.get(server);
+ TabletServerMutations<Mutation> currentMutations = serversMutations.get(server);
if (currentMutations == null) {
serversMutations.put(server, entry.getValue());
@@ -740,8 +740,8 @@ public class TabletServerBatchWriter {
}
}
- private synchronized TabletServerMutations getMutationsToSend(String server) {
- TabletServerMutations tsmuts = serversMutations.remove(server);
+ private synchronized TabletServerMutations<Mutation> getMutationsToSend(String server) {
+ TabletServerMutations<Mutation> tsmuts = serversMutations.remove(server);
if (tsmuts == null)
queued.remove(server);
@@ -759,7 +759,7 @@ public class TabletServerBatchWriter {
@Override
public void run() {
try {
- TabletServerMutations tsmuts = getMutationsToSend(location);
+ TabletServerMutations<Mutation> tsmuts = getMutationsToSend(location);
while (tsmuts != null) {
send(tsmuts);
@@ -772,7 +772,7 @@ public class TabletServerBatchWriter {
}
}
- public void send(TabletServerMutations tsm) throws AccumuloServerException, AccumuloSecurityException {
+ public void send(TabletServerMutations<Mutation> tsm) throws AccumuloServerException, AccumuloSecurityException {
MutationSet failures = null;
@@ -846,6 +846,8 @@ public class TabletServerBatchWriter {
return new MutationSet();
}
TInfo tinfo = Tracer.traceInfo();
+
+ // TODO remove this
TTransport transport = null;
timeoutTracker.startingWrite();
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
index 62518ec..e8cd678 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
@@ -76,8 +76,8 @@ public class TimeoutTabletLocator extends TabletLocator {
}
@Override
- public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials)
- throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+ public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+ TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
try {
locator.binMutations(mutations, binnedMutations, failures, credentials);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index 1179559..4af2ea5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -22,6 +22,8 @@ import org.apache.accumulo.core.client.BatchDeleter;
import org.apache.accumulo.core.client.BatchScanner;
import org.apache.accumulo.core.client.BatchWriter;
import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriterConfig;
import org.apache.accumulo.core.client.Connector;
import org.apache.accumulo.core.client.Instance;
import org.apache.accumulo.core.client.MultiTableBatchWriter;
@@ -128,4 +130,10 @@ public class MockConnector extends Connector {
return new MockInstanceOperations(acu);
}
+ @Override
+ public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException {
+ // TODO add implementation
+ throw new UnsupportedOperationException();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
index aca3ba4..8d91d5d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
@@ -40,10 +40,10 @@ public class MockTabletLocator extends TabletLocator {
}
@Override
- public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
- AccumuloSecurityException, TableNotFoundException {
- TabletServerMutations tsm = new TabletServerMutations();
- for (Mutation m : mutations)
+ public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+ TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+ TabletServerMutations<T> tsm = new TabletServerMutations<T>();
+ for (T m : mutations)
tsm.addMutation(new KeyExtent(), m);
binnedMutations.put("", tsm);
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
index d44a7a6..eaa61b9 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
@@ -17,6 +17,7 @@
package org.apache.accumulo.core.data;
import java.io.Serializable;
+import java.nio.ByteBuffer;
public class ArrayByteSequence extends ByteSequence implements Serializable {
@@ -48,6 +49,18 @@ public class ArrayByteSequence extends ByteSequence implements Serializable {
this(s.getBytes());
}
+ public ArrayByteSequence(ByteBuffer buffer) {
+ if (buffer.hasArray()) {
+ this.data = buffer.array();
+ this.offset = buffer.arrayOffset();
+ this.length = buffer.limit();
+ } else {
+ this.data = new byte[buffer.remaining()];
+ this.offset = 0;
+ buffer.get(data);
+ }
+ }
+
@Override
public byte byteAt(int i) {
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/Condition.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Condition.java b/core/src/main/java/org/apache/accumulo/core/data/Condition.java
new file mode 100644
index 0000000..97df7e0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/Condition.java
@@ -0,0 +1,148 @@
+/*
+ * 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.accumulo.core.data;
+
+import java.util.HashSet;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.hadoop.io.Text;
+
+/**
+ *
+ * @since 1.6.0
+ */
+public class Condition {
+
+ private ByteSequence cf;
+ private ByteSequence cq;
+ private ByteSequence cv;
+ private ByteSequence val;
+ private Long ts;
+ private IteratorSetting iterators[] = new IteratorSetting[0];
+ private static final ByteSequence EMPTY = new ArrayByteSequence(new byte[0]);
+
+
+ public Condition(CharSequence cf, CharSequence cq) {
+ ArgumentChecker.notNull(cf, cq);
+ this.cf = new ArrayByteSequence(cf.toString().getBytes(Constants.UTF8));
+ this.cq = new ArrayByteSequence(cq.toString().getBytes(Constants.UTF8));
+ this.cv = EMPTY;
+ }
+
+ public Condition(byte[] cf, byte[] cq) {
+ ArgumentChecker.notNull(cf, cq);
+ this.cf = new ArrayByteSequence(cf);
+ this.cq = new ArrayByteSequence(cq);
+ this.cv = EMPTY;
+ }
+
+ public Condition(Text cf, Text cq) {
+ ArgumentChecker.notNull(cf, cq);
+ this.cf = new ArrayByteSequence(cf.getBytes(), 0, cf.getLength());
+ this.cq = new ArrayByteSequence(cq.getBytes(), 0, cq.getLength());
+ this.cv = EMPTY;
+ }
+
+ public Condition(ByteSequence cf, ByteSequence cq) {
+ ArgumentChecker.notNull(cf, cq);
+ this.cf = cf;
+ this.cq = cq;
+ this.cv = EMPTY;
+ }
+
+ public ByteSequence getFamily() {
+ return cf;
+ }
+
+ public ByteSequence getQualifier() {
+ return cq;
+ }
+
+ public Condition setTimestamp(long ts) {
+ this.ts = ts;
+ return this;
+ }
+
+ public Long getTimestamp() {
+ return ts;
+ }
+
+ public Condition setValue(CharSequence value) {
+ ArgumentChecker.notNull(value);
+ this.val = new ArrayByteSequence(value.toString().getBytes(Constants.UTF8));
+ return this;
+ }
+
+ public Condition setValue(byte[] value) {
+ ArgumentChecker.notNull(value);
+ this.val = new ArrayByteSequence(value);
+ return this;
+ }
+
+ public Condition setValue(Text value) {
+ ArgumentChecker.notNull(value);
+ this.val = new ArrayByteSequence(value.getBytes(), 0, value.getLength());
+ return this;
+ }
+
+ public Condition setValue(ByteSequence value) {
+ ArgumentChecker.notNull(value);
+ this.val = value;
+ return this;
+ }
+
+ public ByteSequence getValue() {
+ return val;
+ }
+
+ public Condition setVisibility(ColumnVisibility cv) {
+ ArgumentChecker.notNull(cv);
+ this.cv = new ArrayByteSequence(cv.getExpression());
+ return this;
+ }
+
+ public ByteSequence getVisibility() {
+ return cv;
+ }
+
+ public Condition setIterators(IteratorSetting... iterators) {
+ ArgumentChecker.notNull(iterators);
+
+ if (iterators.length > 1) {
+ HashSet<String> names = new HashSet<String>();
+ HashSet<Integer> prios = new HashSet<Integer>();
+
+ for (IteratorSetting iteratorSetting : iterators) {
+ if (!names.add(iteratorSetting.getName()))
+ throw new IllegalArgumentException("iterator name used more than once " + iteratorSetting.getName());
+ if (!prios.add(iteratorSetting.getPriority()))
+ throw new IllegalArgumentException("iterator priority used more than once " + iteratorSetting.getPriority());
+ }
+ }
+
+ this.iterators = iterators;
+ return this;
+ }
+
+ public IteratorSetting[] getIterators() {
+ return iterators;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
new file mode 100644
index 0000000..23bf7d0
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java
@@ -0,0 +1,83 @@
+/*
+ * 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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.hadoop.io.Text;
+
+/**
+ * @since 1.6.0
+ */
+public class ConditionalMutation extends Mutation {
+
+ private List<Condition> conditions = new ArrayList<Condition>();
+
+ public ConditionalMutation(byte[] row, Condition condition, Condition... conditions) {
+ super(row);
+ init(condition, conditions);
+ }
+
+ public ConditionalMutation(byte[] row, int start, int length, Condition condition, Condition... conditions) {
+ super(row, start, length);
+ init(condition, conditions);
+ }
+
+ public ConditionalMutation(Text row, Condition condition, Condition... conditions) {
+ super(row);
+ init(condition, conditions);
+ }
+
+ public ConditionalMutation(CharSequence row, Condition condition, Condition... conditions) {
+ super(row);
+ init(condition, conditions);
+ }
+
+ public ConditionalMutation(ByteSequence row, Condition condition, Condition... conditions) {
+ // TODO add ByteSequence methods to mutations
+ super(row.toArray());
+ init(condition, conditions);
+ }
+
+ public ConditionalMutation(ConditionalMutation cm) {
+ super(cm);
+ this.conditions = new ArrayList<Condition>(cm.conditions);
+ }
+
+ private void init(Condition condition, Condition... conditions) {
+ ArgumentChecker.notNull(condition);
+ this.conditions.add(condition);
+ if (conditions.length > 0) {
+ this.conditions.addAll(Arrays.asList(conditions));
+ }
+ }
+
+ public void addCondition(Condition condition) {
+ ArgumentChecker.notNull(condition);
+ this.conditions.add(condition);
+ }
+
+ public List<Condition> getConditions() {
+ return Collections.unmodifiableList(conditions);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
index 00cefbf..4ac3f0c 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
@@ -27,6 +27,7 @@ import java.util.List;
import org.apache.accumulo.core.data.thrift.TMutation;
import org.apache.accumulo.core.security.ColumnVisibility;
import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
@@ -65,121 +66,8 @@ public class Mutation implements Writable {
private byte[] data;
private int entries;
private List<byte[]> values;
-
- // created this little class instead of using ByteArrayOutput stream and DataOutputStream
- // because both are synchronized... lots of small syncs slow things down
- private static class ByteBuffer {
-
- int offset;
- byte data[] = new byte[64];
-
- private void reserve(int l) {
- if (offset + l > data.length) {
- int newSize = data.length * 2;
- while (newSize <= offset + l)
- newSize = newSize * 2;
-
- byte[] newData = new byte[newSize];
- System.arraycopy(data, 0, newData, 0, offset);
- data = newData;
- }
-
- }
-
- public void add(byte[] bytes, int off, int length) {
- reserve(length);
- System.arraycopy(bytes, off, data, offset, length);
- offset += length;
- }
-
- void add(boolean b) {
- reserve(1);
- if (b)
- data[offset++] = 1;
- else
- data[offset++] = 0;
- }
-
- public byte[] toArray() {
- byte ret[] = new byte[offset];
- System.arraycopy(data, 0, ret, 0, offset);
- return ret;
- }
-
- public void writeVLong(long i) {
- reserve(9);
- if (i >= -112 && i <= 127) {
- data[offset++] = (byte)i;
- return;
- }
-
- int len = -112;
- if (i < 0) {
- i ^= -1L; // take one's complement'
- len = -120;
- }
-
- long tmp = i;
- while (tmp != 0) {
- tmp = tmp >> 8;
- len--;
- }
-
- data[offset++] = (byte)len;
-
- len = (len < -120) ? -(len + 120) : -(len + 112);
-
- for (int idx = len; idx != 0; idx--) {
- int shiftbits = (idx - 1) * 8;
- long mask = 0xFFL << shiftbits;
- data[offset++] = (byte)((i & mask) >> shiftbits);
- }
- }
- }
-
- private static class SimpleReader {
- int offset;
- byte data[];
-
- SimpleReader(byte b[]) {
- this.data = b;
- }
-
- int readInt() {
- return (data[offset++] << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0);
- }
-
- long readLong() {
- return (((long) data[offset++] << 56) + ((long) (data[offset++] & 255) << 48) + ((long) (data[offset++] & 255) << 40)
- + ((long) (data[offset++] & 255) << 32) + ((long) (data[offset++] & 255) << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0));
- }
-
- void readBytes(byte b[]) {
- System.arraycopy(data, offset, b, 0, b.length);
- offset += b.length;
- }
-
- boolean readBoolean() {
- return (data[offset++] == 1);
- }
-
- long readVLong() {
- byte firstByte = data[offset++];
- int len = WritableUtils.decodeVIntSize(firstByte);
- if (len == 1) {
- return firstByte;
- }
- long i = 0;
- for (int idx = 0; idx < len-1; idx++) {
- byte b = data[offset++];
- i = i << 8;
- i = i | (b & 0xFF);
- }
- return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
- }
- }
- private ByteBuffer buffer;
+ private UnsynchronizedBuffer.Writer buffer;
private List<ColumnUpdate> updates;
@@ -205,7 +93,7 @@ public class Mutation implements Writable {
public Mutation(byte[] row, int start, int length) {
this.row = new byte[length];
System.arraycopy(row, start, this.row, 0, length);
- buffer = new ByteBuffer();
+ buffer = new UnsynchronizedBuffer.Writer();
}
public Mutation(Text row) {
@@ -445,7 +333,7 @@ public class Mutation implements Writable {
put(columnFamily, columnQualifier, columnVisibility.getExpression(), true, timestamp, true, EMPTY_BYTES);
}
- private byte[] oldReadBytes(SimpleReader in) {
+ private byte[] oldReadBytes(UnsynchronizedBuffer.Reader in) {
int len = in.readInt();
if (len == 0)
return EMPTY_BYTES;
@@ -455,7 +343,7 @@ public class Mutation implements Writable {
return bytes;
}
- private byte[] readBytes(SimpleReader in) {
+ private byte[] readBytes(UnsynchronizedBuffer.Reader in) {
int len = (int)in.readVLong();
if (len == 0)
return EMPTY_BYTES;
@@ -468,7 +356,7 @@ public class Mutation implements Writable {
public List<ColumnUpdate> getUpdates() {
serialize();
- SimpleReader in = new SimpleReader(data);
+ UnsynchronizedBuffer.Reader in = new UnsynchronizedBuffer.Reader(data);
if (updates == null) {
if (entries == 1) {
@@ -490,7 +378,7 @@ public class Mutation implements Writable {
return new ColumnUpdate(cf, cq, cv, hasts, ts, deleted, val);
}
- private ColumnUpdate deserializeColumnUpdate(SimpleReader in) {
+ private ColumnUpdate deserializeColumnUpdate(UnsynchronizedBuffer.Reader in) {
byte[] cf = readBytes(in);
byte[] cq = readBytes(in);
byte[] cv = readBytes(in);
@@ -623,8 +511,8 @@ public class Mutation implements Writable {
}
// convert data to new format
- SimpleReader din = new SimpleReader(localData);
- buffer = new ByteBuffer();
+ UnsynchronizedBuffer.Reader din = new UnsynchronizedBuffer.Reader(localData);
+ buffer = new UnsynchronizedBuffer.Writer();
for (int i = 0; i < localEntries; i++) {
byte[] cf = oldReadBytes(din);
byte[] cq = oldReadBytes(din);
[2/6] ACCUMULO-1000 added conditional mutations to Accumulo
Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
index ce84054..6f6a9ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
@@ -23,6 +23,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
+import org.apache.accumulo.core.data.ByteSequence;
import org.apache.hadoop.io.Text;
public class ByteBufferUtil {
@@ -76,4 +77,16 @@ public class ByteBufferUtil {
public static String toString(ByteBuffer bytes) {
return new String(bytes.array(), bytes.position(), bytes.remaining());
}
+
+ public static ByteBuffer toByteBuffers(ByteSequence bs) {
+ if (bs == null)
+ return null;
+
+ if (bs.isBackedByArray()) {
+ return ByteBuffer.wrap(bs.getBackingArray(), bs.offset(), bs.length());
+ } else {
+ // TODO create more efficient impl
+ return ByteBuffer.wrap(bs.toArray());
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java b/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
new file mode 100644
index 0000000..b640581
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/UnsynchronizedBuffer.java
@@ -0,0 +1,195 @@
+/*
+ * 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.accumulo.core.util;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ *
+ */
+public class UnsynchronizedBuffer {
+ // created this little class instead of using ByteArrayOutput stream and DataOutputStream
+ // because both are synchronized... lots of small syncs slow things down
+ public static class Writer {
+
+ int offset = 0;
+ byte data[];
+
+ public Writer() {
+ data = new byte[64];
+ }
+
+ public Writer(int initialCapacity) {
+ data = new byte[initialCapacity];
+ }
+
+ private void reserve(int l) {
+ if (offset + l > data.length) {
+ int newSize = UnsynchronizedBuffer.nextArraySize(offset + l);
+
+ byte[] newData = new byte[newSize];
+ System.arraycopy(data, 0, newData, 0, offset);
+ data = newData;
+ }
+
+ }
+
+ public void add(byte[] bytes, int off, int length) {
+ reserve(length);
+ System.arraycopy(bytes, off, data, offset, length);
+ offset += length;
+ }
+
+ public void add(boolean b) {
+ reserve(1);
+ if (b)
+ data[offset++] = 1;
+ else
+ data[offset++] = 0;
+ }
+
+ public byte[] toArray() {
+ byte ret[] = new byte[offset];
+ System.arraycopy(data, 0, ret, 0, offset);
+ return ret;
+ }
+
+ public ByteBuffer toByteBuffer() {
+ return ByteBuffer.wrap(data, 0, offset);
+ }
+
+ public void writeVInt(int i) {
+ writeVLong(i);
+ }
+
+ public void writeVLong(long i) {
+ reserve(9);
+ if (i >= -112 && i <= 127) {
+ data[offset++] = (byte) i;
+ return;
+ }
+
+ int len = -112;
+ if (i < 0) {
+ i ^= -1L; // take one's complement'
+ len = -120;
+ }
+
+ long tmp = i;
+ while (tmp != 0) {
+ tmp = tmp >> 8;
+ len--;
+ }
+
+ data[offset++] = (byte) len;
+
+ len = (len < -120) ? -(len + 120) : -(len + 112);
+
+ for (int idx = len; idx != 0; idx--) {
+ int shiftbits = (idx - 1) * 8;
+ long mask = 0xFFL << shiftbits;
+ data[offset++] = (byte) ((i & mask) >> shiftbits);
+ }
+ }
+ }
+
+ public static class Reader {
+ int offset;
+ byte data[];
+
+ public Reader(byte b[]) {
+ this.data = b;
+ }
+
+ public Reader(ByteBuffer buffer) {
+ if (buffer.hasArray()) {
+ offset = buffer.arrayOffset();
+ data = buffer.array();
+ } else {
+ data = new byte[buffer.remaining()];
+ buffer.get(data);
+ }
+ }
+
+ public int readInt() {
+ return (data[offset++] << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0);
+ }
+
+ public long readLong() {
+ return (((long) data[offset++] << 56) + ((long) (data[offset++] & 255) << 48) + ((long) (data[offset++] & 255) << 40)
+ + ((long) (data[offset++] & 255) << 32) + ((long) (data[offset++] & 255) << 24) + ((data[offset++] & 255) << 16) + ((data[offset++] & 255) << 8) + ((data[offset++] & 255) << 0));
+ }
+
+ public void readBytes(byte b[]) {
+ System.arraycopy(data, offset, b, 0, b.length);
+ offset += b.length;
+ }
+
+ public boolean readBoolean() {
+ return (data[offset++] == 1);
+ }
+
+ public int readVInt() {
+ return (int) readVLong();
+ }
+
+ public long readVLong() {
+ byte firstByte = data[offset++];
+ int len = WritableUtils.decodeVIntSize(firstByte);
+ if (len == 1) {
+ return firstByte;
+ }
+ long i = 0;
+ for (int idx = 0; idx < len - 1; idx++) {
+ byte b = data[offset++];
+ i = i << 8;
+ i = i | (b & 0xFF);
+ }
+ return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+ }
+ }
+
+ /**
+ * Determines what next array size should be by rounding up to next power of two.
+ *
+ */
+ public static int nextArraySize(int i) {
+ if (i < 0)
+ throw new IllegalArgumentException();
+
+ if (i > (1 << 30))
+ return Integer.MAX_VALUE; // this is the next power of 2 minus one... a special case
+
+ if (i == 0) {
+ return 1;
+ }
+
+ // round up to next power of two
+ int ret = i;
+ ret--;
+ ret |= ret >> 1;
+ ret |= ret >> 2;
+ ret |= ret >> 4;
+ ret |= ret >> 8;
+ ret |= ret >> 16;
+ ret++;
+
+ return ret;
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/thrift/data.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/data.thrift b/core/src/main/thrift/data.thrift
index 4e2d3e6..ae6f439 100644
--- a/core/src/main/thrift/data.thrift
+++ b/core/src/main/thrift/data.thrift
@@ -110,10 +110,46 @@ struct UpdateErrors {
3:map<TKeyExtent, client.SecurityErrorCode> authorizationFailures
}
+enum TCMStatus {
+ ACCEPTED,
+ REJECTED,
+ VIOLATED,
+ IGNORED
+}
+
+struct TCMResult {
+ 1:i64 cmid,
+ 2:TCMStatus status
+}
+
struct MapFileInfo {
1:i64 estimatedSize
}
+struct TCondition {
+ 1:binary cf;
+ 2:binary cq;
+ 3:binary cv;
+ 4:i64 ts;
+ 5:bool hasTimestamp;
+ 6:binary val;
+ 7:binary iterators;
+}
+
+struct TConditionalMutation {
+ 1:list<TCondition> conditions;
+ 2:TMutation mutation;
+ 3:i64 id;
+}
+
+struct TConditionalSession {
+ 1:i64 sessionId;
+ 2:string tserverLock;
+ 3:i64 ttl;
+}
+
+typedef map<TKeyExtent,list<TConditionalMutation>> CMBatch
+
typedef map<TKeyExtent,list<TMutation>> UpdateBatch
typedef map<TKeyExtent, map<string, MapFileInfo>> TabletFiles
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/thrift/tabletserver.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index e6adbf4..4f9f13a 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -160,13 +160,21 @@ service TabletClientService extends client.ClientService {
data.UpdateID startUpdate(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec),
oneway void applyUpdates(1:trace.TInfo tinfo, 2:data.UpdateID updateID, 3:data.TKeyExtent keyExtent, 4:list<data.TMutation> mutations),
data.UpdateErrors closeUpdate(2:trace.TInfo tinfo, 1:data.UpdateID updateID) throws (1:NoSuchScanIDException nssi),
-
+
//the following call supports making a single update to a tablet
void update(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:data.TKeyExtent keyExtent, 3:data.TMutation mutation)
throws (1:client.ThriftSecurityException sec,
2:NotServingTabletException nste,
3:ConstraintViolationException cve),
+
+ data.TConditionalSession startConditionalUpdate(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:list<binary> authorizations, 4:string tableID)
+ throws (1:client.ThriftSecurityException sec);
+ list<data.TCMResult> conditionalUpdate(1:trace.TInfo tinfo, 2:data.UpdateID sessID, 3:data.CMBatch mutations, 4:list<string> symbols)
+ throws (1:NoSuchScanIDException nssi);
+ void invalidateConditionalUpdate(1:trace.TInfo tinfo, 2:data.UpdateID sessID);
+ oneway void closeConditionalUpdate(1:trace.TInfo tinfo, 2:data.UpdateID sessID);
+
// on success, returns an empty list
list<data.TKeyExtent> bulkImport(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 4:i64 tid, 2:data.TabletFiles files, 5:bool setTime) throws (1:client.ThriftSecurityException sec),
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 0a34575..fe2f09c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -219,7 +219,7 @@ public class TabletLocatorImplTest extends TestCase {
}
private void runTest(TabletLocatorImpl metaCache, List<Mutation> ml, Map<String,Map<KeyExtent,List<String>>> emb, String... efailures) throws Exception {
- Map<String,TabletServerMutations> binnedMutations = new HashMap<String,TabletServerMutations>();
+ Map<String,TabletServerMutations<Mutation>> binnedMutations = new HashMap<String,TabletServerMutations<Mutation>>();
List<Mutation> afailures = new ArrayList<Mutation>();
metaCache.binMutations(ml, binnedMutations, afailures, credential);
@@ -239,11 +239,11 @@ public class TabletLocatorImplTest extends TestCase {
}
- private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations> actual) {
+ private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations<Mutation>> actual) {
assertEquals(expected.keySet(), actual.keySet());
for (String server : actual.keySet()) {
- TabletServerMutations atb = actual.get(server);
+ TabletServerMutations<Mutation> atb = actual.get(server);
Map<KeyExtent,List<String>> etb = expected.get(server);
assertEquals(etb.keySet(), atb.getMutations().keySet());
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
index bf577c1..1608576 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.PartialKey;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.file.rfile.RelativeKey.MByteSequence;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -39,25 +40,25 @@ public class RelativeKeyTest {
@Test
public void testBasicRelativeKey() {
- assertEquals(1, RelativeKey.nextArraySize(0));
- assertEquals(1, RelativeKey.nextArraySize(1));
- assertEquals(2, RelativeKey.nextArraySize(2));
- assertEquals(4, RelativeKey.nextArraySize(3));
- assertEquals(4, RelativeKey.nextArraySize(4));
- assertEquals(8, RelativeKey.nextArraySize(5));
- assertEquals(8, RelativeKey.nextArraySize(8));
- assertEquals(16, RelativeKey.nextArraySize(9));
+ assertEquals(1, UnsynchronizedBuffer.nextArraySize(0));
+ assertEquals(1, UnsynchronizedBuffer.nextArraySize(1));
+ assertEquals(2, UnsynchronizedBuffer.nextArraySize(2));
+ assertEquals(4, UnsynchronizedBuffer.nextArraySize(3));
+ assertEquals(4, UnsynchronizedBuffer.nextArraySize(4));
+ assertEquals(8, UnsynchronizedBuffer.nextArraySize(5));
+ assertEquals(8, UnsynchronizedBuffer.nextArraySize(8));
+ assertEquals(16, UnsynchronizedBuffer.nextArraySize(9));
- assertEquals(1 << 16, RelativeKey.nextArraySize((1 << 16) - 1));
- assertEquals(1 << 16, RelativeKey.nextArraySize(1 << 16));
- assertEquals(1 << 17, RelativeKey.nextArraySize((1 << 16) + 1));
+ assertEquals(1 << 16, UnsynchronizedBuffer.nextArraySize((1 << 16) - 1));
+ assertEquals(1 << 16, UnsynchronizedBuffer.nextArraySize(1 << 16));
+ assertEquals(1 << 17, UnsynchronizedBuffer.nextArraySize((1 << 16) + 1));
- assertEquals(1 << 30, RelativeKey.nextArraySize((1 << 30) - 1));
+ assertEquals(1 << 30, UnsynchronizedBuffer.nextArraySize((1 << 30) - 1));
- assertEquals(1 << 30, RelativeKey.nextArraySize(1 << 30));
+ assertEquals(1 << 30, UnsynchronizedBuffer.nextArraySize(1 << 30));
- assertEquals(Integer.MAX_VALUE, RelativeKey.nextArraySize(Integer.MAX_VALUE - 1));
- assertEquals(Integer.MAX_VALUE, RelativeKey.nextArraySize(Integer.MAX_VALUE));
+ assertEquals(Integer.MAX_VALUE, UnsynchronizedBuffer.nextArraySize(Integer.MAX_VALUE - 1));
+ assertEquals(Integer.MAX_VALUE, UnsynchronizedBuffer.nextArraySize(Integer.MAX_VALUE));
}
@Test
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java b/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
new file mode 100644
index 0000000..7487ba3
--- /dev/null
+++ b/server/src/main/java/org/apache/accumulo/server/data/ServerConditionalMutation.java
@@ -0,0 +1,58 @@
+/*
+ * 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.accumulo.server.data;
+
+import java.util.List;
+
+import org.apache.accumulo.core.client.impl.Translator;
+import org.apache.accumulo.core.data.thrift.TCondition;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+
+/**
+ *
+ */
+public class ServerConditionalMutation extends ServerMutation {
+
+ public static class TCMTranslator extends Translator<TConditionalMutation,ServerConditionalMutation> {
+ @Override
+ public ServerConditionalMutation translate(TConditionalMutation input) {
+ return new ServerConditionalMutation(input);
+ }
+ }
+
+ public static final TCMTranslator TCMT = new TCMTranslator();
+
+ private long cmid;
+ private List<TCondition> conditions;
+
+ public ServerConditionalMutation(TConditionalMutation input) {
+ super(input.mutation);
+
+ this.cmid = input.id;
+ this.conditions = input.conditions;
+ }
+
+ public long getID() {
+ return cmid;
+ }
+
+ public List<TCondition> getConditions() {
+ return conditions;
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 2b98331..cebc338 100644
--- a/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -328,6 +328,13 @@ public class SecurityOperation {
return hasTablePermission(credentials, table, TablePermission.WRITE, true);
}
+ public boolean canConditionallyUpdate(TCredentials credentials, String tableID, List<ByteBuffer> authorizations) throws ThriftSecurityException {
+
+ authenticate(credentials);
+
+ return hasTablePermission(credentials, tableID, TablePermission.WRITE, true) && hasTablePermission(credentials, tableID, TablePermission.READ, true);
+ }
+
public boolean canSplitTablet(TCredentials credentials, String table) throws ThriftSecurityException {
authenticate(credentials);
return hasSystemPermission(credentials, SystemPermission.ALTER_TABLE, false) || hasSystemPermission(credentials, SystemPermission.SYSTEM, false)
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java
new file mode 100644
index 0000000..c25e729
--- /dev/null
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/ConditionalMutationSet.java
@@ -0,0 +1,91 @@
+/*
+ * 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.accumulo.server.tabletserver;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
+import org.apache.hadoop.io.WritableComparator;
+
+/**
+ *
+ */
+public class ConditionalMutationSet {
+
+ static interface DeferFilter {
+ void defer(List<ServerConditionalMutation> scml, List<ServerConditionalMutation> okMutations, List<ServerConditionalMutation> deferred);
+ }
+
+ static class DuplicateFitler implements DeferFilter {
+ public void defer(List<ServerConditionalMutation> scml, List<ServerConditionalMutation> okMutations, List<ServerConditionalMutation> deferred) {
+ okMutations.add(scml.get(0));
+ for (int i = 1; i < scml.size(); i++) {
+ if (Arrays.equals(scml.get(i - 1).getRow(), scml.get(i).getRow())) {
+ deferred.add(scml.get(i));
+ } else {
+ okMutations.add(scml.get(i));
+ }
+ }
+ }
+ }
+
+ static void defer(Map<KeyExtent,List<ServerConditionalMutation>> updates, Map<KeyExtent,List<ServerConditionalMutation>> deferredMutations, DeferFilter filter) {
+ for (Entry<KeyExtent,List<ServerConditionalMutation>> entry : updates.entrySet()) {
+ List<ServerConditionalMutation> scml = entry.getValue();
+ List<ServerConditionalMutation> okMutations = new ArrayList<ServerConditionalMutation>(scml.size());
+ List<ServerConditionalMutation> deferred = new ArrayList<ServerConditionalMutation>();
+ filter.defer(scml, okMutations, deferred);
+
+ if (deferred.size() > 0) {
+ scml.clear();
+ scml.addAll(okMutations);
+ List<ServerConditionalMutation> l = deferredMutations.get(entry.getKey());
+ if (l == null) {
+ l = deferred;
+ deferredMutations.put(entry.getKey(), l);
+ } else {
+ l.addAll(deferred);
+ }
+
+ }
+ }
+ }
+
+ static void deferDuplicatesRows(Map<KeyExtent,List<ServerConditionalMutation>> updates, Map<KeyExtent,List<ServerConditionalMutation>> deferred) {
+ defer(updates, deferred, new DuplicateFitler());
+ }
+
+ static void sortConditionalMutations(Map<KeyExtent,List<ServerConditionalMutation>> updates) {
+ for (Entry<KeyExtent,List<ServerConditionalMutation>> entry : updates.entrySet()) {
+ // TODO check if its already in sorted order?
+ // TODO maybe the potential benefit of sorting is not worth the cost
+ Collections.sort(entry.getValue(), new Comparator<ServerConditionalMutation>() {
+ @Override
+ public int compare(ServerConditionalMutation o1, ServerConditionalMutation o2) {
+ return WritableComparator.compareBytes(o1.getRow(), 0, o1.getRow().length, o2.getRow(), 0, o2.getRow().length);
+ }
+ });
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java
new file mode 100644
index 0000000..f057ca3
--- /dev/null
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/RowLocks.java
@@ -0,0 +1,162 @@
+/*
+ * 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.accumulo.server.tabletserver;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.accumulo.core.data.ArrayByteSequence;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
+import org.apache.accumulo.server.tabletserver.ConditionalMutationSet.DeferFilter;
+
+/**
+ *
+ */
+class RowLocks {
+
+ private Map<ByteSequence,RowLock> rowLocks = new HashMap<ByteSequence,RowLock>();
+
+ static class RowLock {
+ ReentrantLock rlock;
+ int count;
+ ByteSequence rowSeq;
+
+ RowLock(ReentrantLock rlock, ByteSequence rowSeq) {
+ this.rlock = rlock;
+ this.count = 0;
+ this.rowSeq = rowSeq;
+ }
+
+ public boolean tryLock() {
+ return rlock.tryLock();
+ }
+
+ public void lock() {
+ rlock.lock();
+ }
+
+ public void unlock() {
+ rlock.unlock();
+ }
+ }
+
+ private RowLock getRowLock(ArrayByteSequence rowSeq) {
+ RowLock lock = rowLocks.get(rowSeq);
+ if (lock == null) {
+ lock = new RowLock(new ReentrantLock(), rowSeq);
+ rowLocks.put(rowSeq, lock);
+ }
+
+ lock.count++;
+ return lock;
+ }
+
+ private void returnRowLock(RowLock lock) {
+ if (lock.count == 0)
+ throw new IllegalStateException();
+ lock.count--;
+
+ if (lock.count == 0) {
+ rowLocks.remove(lock.rowSeq);
+ }
+ }
+
+ List<RowLock> acquireRowlocks(Map<KeyExtent,List<ServerConditionalMutation>> updates, Map<KeyExtent,List<ServerConditionalMutation>> deferred) {
+ ArrayList<RowLock> locks = new ArrayList<RowLock>();
+
+ // assume that mutations are in sorted order to avoid deadlock
+ synchronized (rowLocks) {
+ for (List<ServerConditionalMutation> scml : updates.values()) {
+ for (ServerConditionalMutation scm : scml) {
+ locks.add(getRowLock(new ArrayByteSequence(scm.getRow())));
+ }
+ }
+ }
+
+ HashSet<ByteSequence> rowsNotLocked = null;
+
+ // acquire as many locks as possible, not blocking on rows that are already locked
+ if (locks.size() > 1) {
+ for (RowLock rowLock : locks) {
+ if (!rowLock.tryLock()) {
+ if (rowsNotLocked == null)
+ rowsNotLocked = new HashSet<ByteSequence>();
+ rowsNotLocked.add(rowLock.rowSeq);
+ }
+ }
+ } else {
+ // if there is only one lock, then wait for it
+ locks.get(0).lock();
+ }
+
+ if (rowsNotLocked != null) {
+
+ final HashSet<ByteSequence> rnlf = rowsNotLocked;
+ // assume will get locks needed, do something expensive otherwise
+ ConditionalMutationSet.defer(updates, deferred, new DeferFilter() {
+ @Override
+ public void defer(List<ServerConditionalMutation> scml, List<ServerConditionalMutation> okMutations, List<ServerConditionalMutation> deferred) {
+ for (ServerConditionalMutation scm : scml) {
+ if (rnlf.contains(new ArrayByteSequence(scm.getRow())))
+ deferred.add(scm);
+ else
+ okMutations.add(scm);
+
+ }
+ }
+ });
+
+ ArrayList<RowLock> filteredLocks = new ArrayList<RowLock>();
+ ArrayList<RowLock> locksToReturn = new ArrayList<RowLock>();
+ for (RowLock rowLock : locks) {
+ if (rowsNotLocked.contains(rowLock.rowSeq)) {
+ locksToReturn.add(rowLock);
+ } else {
+ filteredLocks.add(rowLock);
+ }
+ }
+
+ synchronized (rowLocks) {
+ for (RowLock rowLock : locksToReturn) {
+ returnRowLock(rowLock);
+ }
+ }
+
+ locks = filteredLocks;
+ }
+ return locks;
+ }
+
+ void releaseRowLocks(List<RowLock> locks) {
+ for (RowLock rowLock : locks) {
+ rowLock.unlock();
+ }
+
+ synchronized (rowLocks) {
+ for (RowLock rowLock : locks) {
+ returnRowLock(rowLock);
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
index e9b973a..01b0dc2 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
@@ -1651,7 +1651,7 @@ public class Tablet {
}
}
- private Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, int num, HashSet<Column> columns) throws IOException {
+ private Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, int num, Set<Column> columns) throws IOException {
// log.info("In nextBatch..");
@@ -1739,7 +1739,7 @@ public class Tablet {
public long numBytes;
}
- Scanner createScanner(Range range, int num, HashSet<Column> columns, Authorizations authorizations, List<IterInfo> ssiList,
+ Scanner createScanner(Range range, int num, Set<Column> columns, Authorizations authorizations, List<IterInfo> ssiList,
Map<String,Map<String,String>> ssio, boolean isolated, AtomicBoolean interruptFlag) {
// do a test to see if this range falls within the tablet, if it does not
// then clip will throw an exception
@@ -1873,14 +1873,14 @@ public class Tablet {
// scan options
Authorizations authorizations;
byte[] defaultLabels;
- HashSet<Column> columnSet;
+ Set<Column> columnSet;
List<IterInfo> ssiList;
Map<String,Map<String,String>> ssio;
AtomicBoolean interruptFlag;
int num;
boolean isolated;
- ScanOptions(int num, Authorizations authorizations, byte[] defaultLabels, HashSet<Column> columnSet, List<IterInfo> ssiList,
+ ScanOptions(int num, Authorizations authorizations, byte[] defaultLabels, Set<Column> columnSet, List<IterInfo> ssiList,
Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag, boolean isolated) {
this.num = num;
this.authorizations = authorizations;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index 7425fed..ccb95fc 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -65,6 +65,8 @@ import org.apache.accumulo.core.Constants;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.CompressedIterators;
+import org.apache.accumulo.core.client.impl.CompressedIterators.IterConfig;
import org.apache.accumulo.core.client.impl.ScannerImpl;
import org.apache.accumulo.core.client.impl.TabletType;
import org.apache.accumulo.core.client.impl.Translator;
@@ -87,7 +89,12 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
import org.apache.accumulo.core.data.thrift.MapFileInfo;
import org.apache.accumulo.core.data.thrift.MultiScanResult;
import org.apache.accumulo.core.data.thrift.ScanResult;
+import org.apache.accumulo.core.data.thrift.TCMResult;
+import org.apache.accumulo.core.data.thrift.TCMStatus;
import org.apache.accumulo.core.data.thrift.TColumn;
+import org.apache.accumulo.core.data.thrift.TCondition;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+import org.apache.accumulo.core.data.thrift.TConditionalSession;
import org.apache.accumulo.core.data.thrift.TKey;
import org.apache.accumulo.core.data.thrift.TKeyExtent;
import org.apache.accumulo.core.data.thrift.TKeyValue;
@@ -140,6 +147,7 @@ import org.apache.accumulo.server.client.ClientServiceHandler;
import org.apache.accumulo.server.client.HdfsZooInstance;
import org.apache.accumulo.server.conf.ServerConfiguration;
import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.data.ServerConditionalMutation;
import org.apache.accumulo.server.data.ServerMutation;
import org.apache.accumulo.server.fs.FileRef;
import org.apache.accumulo.server.fs.VolumeManager;
@@ -158,6 +166,7 @@ import org.apache.accumulo.server.security.AuditedSecurityOperation;
import org.apache.accumulo.server.security.SecurityOperation;
import org.apache.accumulo.server.security.SystemCredentials;
import org.apache.accumulo.server.tabletserver.Compactor.CompactionInfo;
+import org.apache.accumulo.server.tabletserver.RowLocks.RowLock;
import org.apache.accumulo.server.tabletserver.Tablet.CommitSession;
import org.apache.accumulo.server.tabletserver.Tablet.KVEntry;
import org.apache.accumulo.server.tabletserver.Tablet.LookupResult;
@@ -336,12 +345,13 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
SecureRandom random;
Map<Long,Session> sessions;
+ long maxIdle;
SessionManager(AccumuloConfiguration conf) {
random = new SecureRandom();
sessions = new HashMap<Long,Session>();
- final long maxIdle = conf.getTimeInMillis(Property.TSERV_SESSION_MAXIDLE);
+ maxIdle = conf.getTimeInMillis(Property.TSERV_SESSION_MAXIDLE);
Runnable r = new Runnable() {
@Override
@@ -369,6 +379,10 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
return sid;
}
+ long getMaxIdleTime() {
+ return maxIdle;
+ }
+
/**
* while a session is reserved, it cannot be canceled or removed
*
@@ -387,9 +401,30 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
}
+ synchronized Session reserveSession(long sessionId, boolean wait) {
+ Session session = sessions.get(sessionId);
+ if (session != null) {
+ while(wait && session.reserved){
+ try {
+ wait(1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException();
+ }
+ }
+
+ if (session.reserved)
+ throw new IllegalStateException();
+ session.reserved = true;
+ }
+
+ return session;
+
+ }
+
synchronized void unreserveSession(Session session) {
if (!session.reserved)
throw new IllegalStateException();
+ notifyAll();
session.reserved = false;
session.lastAccessTime = System.currentTimeMillis();
}
@@ -399,7 +434,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
if (session != null)
unreserveSession(session);
}
-
+
synchronized Session getSession(long sessionId) {
Session session = sessions.get(sessionId);
if (session != null)
@@ -408,9 +443,15 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
}
Session removeSession(long sessionId) {
+ return removeSession(sessionId, false);
+ }
+
+ Session removeSession(long sessionId, boolean unreserve) {
Session session = null;
synchronized (this) {
session = sessions.remove(sessionId);
+ if(unreserve && session != null)
+ unreserveSession(session);
}
// do clean up out side of lock..
@@ -709,6 +750,18 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
}
+ private static class ConditionalSession extends Session {
+ public TCredentials credentials;
+ public Authorizations auths;
+ public String tableId;
+ public AtomicBoolean interruptFlag;
+
+ @Override
+ public void cleanup() {
+ interruptFlag.set(true);
+ }
+ }
+
private static class UpdateSession extends Session {
public Tablet currentTablet;
public MapCounter<Tablet> successfulCommits = new MapCounter<Tablet>();
@@ -858,6 +911,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
WriteTracker writeTracker = new WriteTracker();
+ private RowLocks rowLocks = new RowLocks();
+
ThriftClientHandler() {
super(instance, watcher);
log.debug(ThriftClientHandler.class.getName() + " created");
@@ -1687,6 +1742,250 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
writeTracker.finishWrite(opid);
}
}
+
+ private void checkConditions(Map<KeyExtent,List<ServerConditionalMutation>> updates, ArrayList<TCMResult> results, ConditionalSession cs,
+ List<String> symbols) throws IOException {
+ Iterator<Entry<KeyExtent,List<ServerConditionalMutation>>> iter = updates.entrySet().iterator();
+
+ CompressedIterators compressedIters = new CompressedIterators(symbols);
+
+ while (iter.hasNext()) {
+ Entry<KeyExtent,List<ServerConditionalMutation>> entry = iter.next();
+ Tablet tablet = onlineTablets.get(entry.getKey());
+
+ if (tablet == null || tablet.isClosed()) {
+ for (ServerConditionalMutation scm : entry.getValue())
+ results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+ iter.remove();
+ } else {
+ List<ServerConditionalMutation> okMutations = new ArrayList<ServerConditionalMutation>(entry.getValue().size());
+
+ for (ServerConditionalMutation scm : entry.getValue()) {
+ if (checkCondition(results, cs, compressedIters, tablet, scm))
+ okMutations.add(scm);
+ }
+
+ entry.setValue(okMutations);
+ }
+
+ }
+ }
+
+ boolean checkCondition(ArrayList<TCMResult> results, ConditionalSession cs, CompressedIterators compressedIters,
+ Tablet tablet, ServerConditionalMutation scm) throws IOException {
+ boolean add = true;
+
+ Set<Column> emptyCols = Collections.emptySet();
+
+ for(TCondition tc : scm.getConditions()){
+
+ Range range;
+ if (tc.hasTimestamp)
+ range = Range.exact(new Text(scm.getRow()), new Text(tc.getCf()), new Text(tc.getCq()), new Text(tc.getCv()), tc.getTs());
+ else
+ range = Range.exact(new Text(scm.getRow()), new Text(tc.getCf()), new Text(tc.getCq()), new Text(tc.getCv()));
+
+ IterConfig ic = compressedIters.decompress(tc.iterators);
+
+ Scanner scanner = tablet.createScanner(range, 1, emptyCols, cs.auths, ic.ssiList, ic.ssio, false, cs.interruptFlag);
+
+ try {
+ ScanBatch batch = scanner.read();
+
+ Value val = null;
+
+ for (KVEntry entry2 : batch.results) {
+ val = entry2.getValue();
+ break;
+ }
+
+ if ((val == null ^ tc.getVal() == null) || (val != null && !Arrays.equals(tc.getVal(), val.get()))) {
+ results.add(new TCMResult(scm.getID(), TCMStatus.REJECTED));
+ add = false;
+ break;
+ }
+
+ } catch (TabletClosedException e) {
+ results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+ add = false;
+ break;
+ } catch (IterationInterruptedException iie) {
+ results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+ add = false;
+ break;
+ } catch (TooManyFilesException tmfe) {
+ results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+ add = false;
+ break;
+ }
+ }
+ return add;
+ }
+
+ private void writeConditionalMutations(Map<KeyExtent,List<ServerConditionalMutation>> updates, ArrayList<TCMResult> results, ConditionalSession sess) {
+ Set<Entry<KeyExtent,List<ServerConditionalMutation>>> es = updates.entrySet();
+
+ Map<CommitSession,List<Mutation>> sendables = new HashMap<CommitSession,List<Mutation>>();
+
+ boolean sessionCanceled = sess.interruptFlag.get();
+
+ for (Entry<KeyExtent,List<ServerConditionalMutation>> entry : es) {
+ Tablet tablet = onlineTablets.get(entry.getKey());
+ if (tablet == null || tablet.isClosed() || sessionCanceled) {
+ for (ServerConditionalMutation scm : entry.getValue())
+ results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+ } else {
+ try {
+
+ List<Mutation> mutations = (List<Mutation>) (List<? extends Mutation>) entry.getValue();
+ if (mutations.size() > 0) {
+
+ CommitSession cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(security, sess.credentials), mutations);
+
+ if (cs == null) {
+ for (ServerConditionalMutation scm : entry.getValue())
+ results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
+ } else {
+ for (ServerConditionalMutation scm : entry.getValue())
+ results.add(new TCMResult(scm.getID(), TCMStatus.ACCEPTED));
+ sendables.put(cs, mutations);
+ }
+ }
+ } catch (TConstraintViolationException e) {
+ if (e.getNonViolators().size() > 0) {
+ sendables.put(e.getCommitSession(), e.getNonViolators());
+ for (Mutation m : e.getNonViolators())
+ results.add(new TCMResult(((ServerConditionalMutation) m).getID(), TCMStatus.ACCEPTED));
+ }
+
+ for (Mutation m : e.getViolators())
+ results.add(new TCMResult(((ServerConditionalMutation) m).getID(), TCMStatus.VIOLATED));
+ }
+ }
+ }
+
+ while (true && sendables.size() > 0) {
+ try {
+ logger.logManyTablets(sendables);
+ break;
+ } catch (IOException ex) {
+ log.warn("logging mutations failed, retrying");
+ } catch (FSError ex) { // happens when DFS is localFS
+ log.warn("logging mutations failed, retrying");
+ } catch (Throwable t) {
+ log.error("Unknown exception logging mutations, counts for mutations in flight not decremented!", t);
+ throw new RuntimeException(t);
+ }
+ }
+
+ for (Entry<CommitSession,? extends List<Mutation>> entry : sendables.entrySet()) {
+ CommitSession commitSession = entry.getKey();
+ List<Mutation> mutations = entry.getValue();
+
+ commitSession.commit(mutations);
+ }
+
+ }
+
+ private Map<KeyExtent,List<ServerConditionalMutation>> conditionalUpdate(ConditionalSession cs, Map<KeyExtent,List<ServerConditionalMutation>> updates,
+ ArrayList<TCMResult> results, List<String> symbols) throws IOException {
+ // sort each list of mutations, this is done to avoid deadlock and doing seeks in order is more efficient and detect duplicate rows.
+ ConditionalMutationSet.sortConditionalMutations(updates);
+
+ Map<KeyExtent,List<ServerConditionalMutation>> deferred = new HashMap<KeyExtent,List<ServerConditionalMutation>>();
+
+ // can not process two mutations for the same row, because one will not see what the other writes
+ ConditionalMutationSet.deferDuplicatesRows(updates, deferred);
+
+ // get as many locks as possible w/o blocking... defer any rows that are locked
+ List<RowLock> locks = rowLocks.acquireRowlocks(updates, deferred);
+ try {
+ checkConditions(updates, results, cs, symbols);
+ writeConditionalMutations(updates, results, cs);
+ } finally {
+ rowLocks.releaseRowLocks(locks);
+ }
+ return deferred;
+ }
+
+ @Override
+ public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableID)
+ throws ThriftSecurityException, TException {
+
+ Authorizations userauths = null;
+ if (!security.canConditionallyUpdate(credentials, tableID, authorizations))
+ throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+ userauths = security.getUserAuthorizations(credentials);
+ for (ByteBuffer auth : authorizations)
+ if (!userauths.contains(ByteBufferUtil.toBytes(auth)))
+ throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
+
+ ConditionalSession cs = new ConditionalSession();
+ cs.auths = new Authorizations(authorizations);
+ cs.credentials = credentials;
+ cs.tableId = tableID;
+ cs.interruptFlag = new AtomicBoolean();
+
+ long sid = sessionManager.createSession(cs, false);
+ return new TConditionalSession(sid, lockID, sessionManager.getMaxIdleTime());
+ }
+
+ @Override
+ public List<TCMResult> conditionalUpdate(TInfo tinfo, long sessID, Map<TKeyExtent,List<TConditionalMutation>> mutations, List<String> symbols)
+ throws NoSuchScanIDException, TException {
+
+ ConditionalSession cs = (ConditionalSession) sessionManager.reserveSession(sessID);
+
+ if (cs == null || cs.interruptFlag.get())
+ throw new NoSuchScanIDException();
+
+ Text tid = new Text(cs.tableId);
+ long opid = writeTracker.startWrite(TabletType.type(new KeyExtent(tid, null, null)));
+
+ try{
+ Map<KeyExtent,List<ServerConditionalMutation>> updates = Translator.translate(mutations, Translator.TKET,
+ new Translator.ListTranslator<TConditionalMutation,ServerConditionalMutation>(ServerConditionalMutation.TCMT));
+
+ for(KeyExtent ke : updates.keySet())
+ if(!ke.getTableId().equals(tid))
+ throw new IllegalArgumentException("Unexpected table id "+tid+" != "+ke.getTableId());
+
+ ArrayList<TCMResult> results = new ArrayList<TCMResult>();
+
+ Map<KeyExtent,List<ServerConditionalMutation>> deferred = conditionalUpdate(cs, updates, results, symbols);
+
+ while (deferred.size() > 0) {
+ deferred = conditionalUpdate(cs, deferred, results, symbols);
+ }
+
+ return results;
+ } catch (IOException ioe) {
+ throw new TException(ioe);
+ }finally{
+ writeTracker.finishWrite(opid);
+ sessionManager.unreserveSession(sessID);
+ }
+ }
+
+ @Override
+ public void invalidateConditionalUpdate(TInfo tinfo, long sessID) throws TException {
+ //this method should wait for any running conditional update to complete
+ //after this method returns a conditional update should not be able to start
+
+ ConditionalSession cs = (ConditionalSession) sessionManager.getSession(sessID);
+ if (cs != null)
+ cs.interruptFlag.set(true);
+
+ cs = (ConditionalSession) sessionManager.reserveSession(sessID, true);
+ if(cs != null)
+ sessionManager.removeSession(sessID, true);
+ }
+
+ @Override
+ public void closeConditionalUpdate(TInfo tinfo, long sessID) throws TException {
+ sessionManager.removeSession(sessID, false);
+ }
@Override
public void splitTablet(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint) throws NotServingTabletException,
@@ -2584,6 +2883,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
private DistributedWorkQueue bulkFailedCopyQ;
+ private String lockID;
+
private static final String METRICS_PREFIX = "tserver";
private static ObjectName OBJECT_NAME = null;
@@ -2705,6 +3006,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
if (tabletServerLock.tryLock(lw, lockContent)) {
log.debug("Obtained tablet server lock " + tabletServerLock.getLockPath());
+ lockID = tabletServerLock.getLockID().serialize(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/");
return;
}
log.info("Waiting for tablet server lock");
@@ -2735,7 +3037,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
}
clientAddress = new InetSocketAddress(clientAddress.getHostName(), clientPort);
announceExistence();
-
+
ThreadPoolExecutor distWorkQThreadPool = new SimpleThreadPool(getSystemConfiguration().getCount(Property.TSERV_WORKQ_THREADS), "distributed work queue");
bulkFailedCopyQ = new DistributedWorkQueue(ZooUtil.getRoot(instance) + Constants.ZBULK_FAILED_COPYQ);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
index 451a079..205cebc 100644
--- a/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ b/server/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
@@ -23,8 +23,6 @@ import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
-import org.junit.Assert;
-
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.TableNotFoundException;
@@ -44,6 +42,7 @@ import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
+import org.junit.Assert;
import org.junit.Test;
public class BulkImporterTest {
@@ -67,8 +66,8 @@ public class BulkImporterTest {
}
@Override
- public void binMutations(List<Mutation> mutations, Map<String,TabletServerMutations> binnedMutations, List<Mutation> failures, TCredentials credentials) throws AccumuloException,
- AccumuloSecurityException, TableNotFoundException {
+ public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures,
+ TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
throw new NotImplementedException();
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java b/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java
new file mode 100644
index 0000000..7e7480f
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java
@@ -0,0 +1,81 @@
+/*
+ * 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.accumulo.test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.data.ConditionalMutation;
+
+
+/**
+ * A writer that will sometimes return unknown. When it returns unknown the condition may or may not have been written.
+ */
+public class FaultyConditionalWriter implements ConditionalWriter {
+
+ private ConditionalWriter cw;
+ private double up;
+ private Random rand;
+ private double wp;
+
+ public FaultyConditionalWriter(ConditionalWriter cw, double unknownProbability, double writeProbability) {
+ this.cw = cw;
+ this.up = unknownProbability;
+ this.wp = writeProbability;
+ this.rand = new Random();
+
+ }
+
+ public Iterator<Result> write(Iterator<ConditionalMutation> mutations) {
+ ArrayList<Result> resultList = new ArrayList<Result>();
+ ArrayList<ConditionalMutation> writes = new ArrayList<ConditionalMutation>();
+
+ while (mutations.hasNext()) {
+ ConditionalMutation cm = mutations.next();
+ if (rand.nextDouble() <= up && rand.nextDouble() > wp)
+ resultList.add(new Result(Status.UNKNOWN, cm, null));
+ else
+ writes.add(cm);
+ }
+
+ if (writes.size() > 0) {
+ Iterator<Result> results = cw.write(writes.iterator());
+
+ while (results.hasNext()) {
+ Result result = results.next();
+
+ if (rand.nextDouble() <= up && rand.nextDouble() <= wp)
+ result = new Result(Status.UNKNOWN, result.getMutation(), result.getTabletServer());
+ resultList.add(result);
+ }
+ }
+ return resultList.iterator();
+ }
+
+ public Result write(ConditionalMutation mutation) {
+ return write(Collections.singleton(mutation).iterator()).next();
+ }
+
+ @Override
+ public void close() {
+ cw.close();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
index e2db273..1c62720 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BadIterator.java
@@ -30,6 +30,11 @@ public class BadIterator extends WrappingIterator {
}
@Override
+ public boolean hasTop() {
+ throw new NullPointerException();
+ }
+
+ @Override
public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
throw new UnsupportedOperationException();
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
index a71b1ad..03eaefb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
@@ -17,10 +17,13 @@
package org.apache.accumulo.test.functional;
import java.io.IOException;
+import java.util.Collection;
import java.util.Map;
import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -30,13 +33,19 @@ import org.apache.accumulo.core.util.UtilWaitThread;
public class SlowIterator extends WrappingIterator {
static private final String SLEEP_TIME = "sleepTime";
+ static private final String SEEK_SLEEP_TIME = "seekSleepTime";
- long sleepTime;
+ private long sleepTime = 0;
+ private long seekSleepTime = 0;
public static void setSleepTime(IteratorSetting is, long millis) {
is.addOption(SLEEP_TIME, Long.toString(millis));
}
+ public static void setSeekSleepTime(IteratorSetting is, long t) {
+ is.addOption(SEEK_SLEEP_TIME, Long.toString(t));
+ }
+
@Override
public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
throw new UnsupportedOperationException();
@@ -49,9 +58,20 @@ public class SlowIterator extends WrappingIterator {
}
@Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+ UtilWaitThread.sleep(seekSleepTime);
+ super.seek(range, columnFamilies, inclusive);
+ }
+
+ @Override
public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
super.init(source, options, env);
- sleepTime = Long.parseLong(options.get(SLEEP_TIME));
+ if (options.containsKey(SLEEP_TIME))
+ sleepTime = Long.parseLong(options.get(SLEEP_TIME));
+
+ if (options.containsKey(SEEK_SLEEP_TIME))
+ seekSleepTime = Long.parseLong(options.get(SEEK_SLEEP_TIME));
}
+
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index e33603f..c84fd7f 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -40,7 +40,10 @@ import org.apache.accumulo.core.data.thrift.IterInfo;
import org.apache.accumulo.core.data.thrift.MapFileInfo;
import org.apache.accumulo.core.data.thrift.MultiScanResult;
import org.apache.accumulo.core.data.thrift.ScanResult;
+import org.apache.accumulo.core.data.thrift.TCMResult;
import org.apache.accumulo.core.data.thrift.TColumn;
+import org.apache.accumulo.core.data.thrift.TConditionalMutation;
+import org.apache.accumulo.core.data.thrift.TConditionalSession;
import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
import org.apache.accumulo.core.data.thrift.TKeyExtent;
import org.apache.accumulo.core.data.thrift.TMutation;
@@ -50,6 +53,7 @@ import org.apache.accumulo.core.master.thrift.TabletServerStatus;
import org.apache.accumulo.core.security.thrift.TCredentials;
import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
@@ -200,6 +204,25 @@ public class NullTserver {
public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
return new ArrayList<ActiveCompaction>();
}
+
+ @Override
+ public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableID)
+ throws ThriftSecurityException,
+ TException {
+ return null;
+ }
+
+ @Override
+ public List<TCMResult> conditionalUpdate(TInfo tinfo, long sessID, Map<TKeyExtent,List<TConditionalMutation>> mutations, List<String> symbols)
+ throws NoSuchScanIDException, TException {
+ return null;
+ }
+
+ @Override
+ public void invalidateConditionalUpdate(TInfo tinfo, long sessID) throws TException {}
+
+ @Override
+ public void closeConditionalUpdate(TInfo tinfo, long sessID) throws TException {}
}
static class Opts extends Help {
[4/6] ACCUMULO-1000 added conditional mutations to Accumulo
Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java
new file mode 100644
index 0000000..b3bfa15
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/thrift/TConditionalSession.java
@@ -0,0 +1,578 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.accumulo.core.data.thrift;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TConditionalSession implements org.apache.thrift.TBase<TConditionalSession, TConditionalSession._Fields>, java.io.Serializable, Cloneable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TConditionalSession");
+
+ private static final org.apache.thrift.protocol.TField SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.I64, (short)1);
+ private static final org.apache.thrift.protocol.TField TSERVER_LOCK_FIELD_DESC = new org.apache.thrift.protocol.TField("tserverLock", org.apache.thrift.protocol.TType.STRING, (short)2);
+ private static final org.apache.thrift.protocol.TField TTL_FIELD_DESC = new org.apache.thrift.protocol.TField("ttl", org.apache.thrift.protocol.TType.I64, (short)3);
+
+ private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TConditionalSessionStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TConditionalSessionTupleSchemeFactory());
+ }
+
+ public long sessionId; // required
+ public String tserverLock; // required
+ public long ttl; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SESSION_ID((short)1, "sessionId"),
+ TSERVER_LOCK((short)2, "tserverLock"),
+ TTL((short)3, "ttl");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // SESSION_ID
+ return SESSION_ID;
+ case 2: // TSERVER_LOCK
+ return TSERVER_LOCK;
+ case 3: // TTL
+ return TTL;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __SESSIONID_ISSET_ID = 0;
+ private static final int __TTL_ISSET_ID = 1;
+ private byte __isset_bitfield = 0;
+ public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+ tmpMap.put(_Fields.TSERVER_LOCK, new org.apache.thrift.meta_data.FieldMetaData("tserverLock", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+ tmpMap.put(_Fields.TTL, new org.apache.thrift.meta_data.FieldMetaData("ttl", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TConditionalSession.class, metaDataMap);
+ }
+
+ public TConditionalSession() {
+ }
+
+ public TConditionalSession(
+ long sessionId,
+ String tserverLock,
+ long ttl)
+ {
+ this();
+ this.sessionId = sessionId;
+ setSessionIdIsSet(true);
+ this.tserverLock = tserverLock;
+ this.ttl = ttl;
+ setTtlIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public TConditionalSession(TConditionalSession other) {
+ __isset_bitfield = other.__isset_bitfield;
+ this.sessionId = other.sessionId;
+ if (other.isSetTserverLock()) {
+ this.tserverLock = other.tserverLock;
+ }
+ this.ttl = other.ttl;
+ }
+
+ public TConditionalSession deepCopy() {
+ return new TConditionalSession(this);
+ }
+
+ @Override
+ public void clear() {
+ setSessionIdIsSet(false);
+ this.sessionId = 0;
+ this.tserverLock = null;
+ setTtlIsSet(false);
+ this.ttl = 0;
+ }
+
+ public long getSessionId() {
+ return this.sessionId;
+ }
+
+ public TConditionalSession setSessionId(long sessionId) {
+ this.sessionId = sessionId;
+ setSessionIdIsSet(true);
+ return this;
+ }
+
+ public void unsetSessionId() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+ }
+
+ /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */
+ public boolean isSetSessionId() {
+ return EncodingUtils.testBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+ }
+
+ public void setSessionIdIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SESSIONID_ISSET_ID, value);
+ }
+
+ public String getTserverLock() {
+ return this.tserverLock;
+ }
+
+ public TConditionalSession setTserverLock(String tserverLock) {
+ this.tserverLock = tserverLock;
+ return this;
+ }
+
+ public void unsetTserverLock() {
+ this.tserverLock = null;
+ }
+
+ /** Returns true if field tserverLock is set (has been assigned a value) and false otherwise */
+ public boolean isSetTserverLock() {
+ return this.tserverLock != null;
+ }
+
+ public void setTserverLockIsSet(boolean value) {
+ if (!value) {
+ this.tserverLock = null;
+ }
+ }
+
+ public long getTtl() {
+ return this.ttl;
+ }
+
+ public TConditionalSession setTtl(long ttl) {
+ this.ttl = ttl;
+ setTtlIsSet(true);
+ return this;
+ }
+
+ public void unsetTtl() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TTL_ISSET_ID);
+ }
+
+ /** Returns true if field ttl is set (has been assigned a value) and false otherwise */
+ public boolean isSetTtl() {
+ return EncodingUtils.testBit(__isset_bitfield, __TTL_ISSET_ID);
+ }
+
+ public void setTtlIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TTL_ISSET_ID, value);
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case SESSION_ID:
+ if (value == null) {
+ unsetSessionId();
+ } else {
+ setSessionId((Long)value);
+ }
+ break;
+
+ case TSERVER_LOCK:
+ if (value == null) {
+ unsetTserverLock();
+ } else {
+ setTserverLock((String)value);
+ }
+ break;
+
+ case TTL:
+ if (value == null) {
+ unsetTtl();
+ } else {
+ setTtl((Long)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SESSION_ID:
+ return Long.valueOf(getSessionId());
+
+ case TSERVER_LOCK:
+ return getTserverLock();
+
+ case TTL:
+ return Long.valueOf(getTtl());
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SESSION_ID:
+ return isSetSessionId();
+ case TSERVER_LOCK:
+ return isSetTserverLock();
+ case TTL:
+ return isSetTtl();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TConditionalSession)
+ return this.equals((TConditionalSession)that);
+ return false;
+ }
+
+ public boolean equals(TConditionalSession that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_sessionId = true;
+ boolean that_present_sessionId = true;
+ if (this_present_sessionId || that_present_sessionId) {
+ if (!(this_present_sessionId && that_present_sessionId))
+ return false;
+ if (this.sessionId != that.sessionId)
+ return false;
+ }
+
+ boolean this_present_tserverLock = true && this.isSetTserverLock();
+ boolean that_present_tserverLock = true && that.isSetTserverLock();
+ if (this_present_tserverLock || that_present_tserverLock) {
+ if (!(this_present_tserverLock && that_present_tserverLock))
+ return false;
+ if (!this.tserverLock.equals(that.tserverLock))
+ return false;
+ }
+
+ boolean this_present_ttl = true;
+ boolean that_present_ttl = true;
+ if (this_present_ttl || that_present_ttl) {
+ if (!(this_present_ttl && that_present_ttl))
+ return false;
+ if (this.ttl != that.ttl)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(TConditionalSession other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ TConditionalSession typedOther = (TConditionalSession)other;
+
+ lastComparison = Boolean.valueOf(isSetSessionId()).compareTo(typedOther.isSetSessionId());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSessionId()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, typedOther.sessionId);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetTserverLock()).compareTo(typedOther.isSetTserverLock());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTserverLock()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tserverLock, typedOther.tserverLock);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetTtl()).compareTo(typedOther.isSetTtl());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTtl()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ttl, typedOther.ttl);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("TConditionalSession(");
+ boolean first = true;
+
+ sb.append("sessionId:");
+ sb.append(this.sessionId);
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("tserverLock:");
+ if (this.tserverLock == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tserverLock);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("ttl:");
+ sb.append(this.ttl);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TConditionalSessionStandardSchemeFactory implements SchemeFactory {
+ public TConditionalSessionStandardScheme getScheme() {
+ return new TConditionalSessionStandardScheme();
+ }
+ }
+
+ private static class TConditionalSessionStandardScheme extends StandardScheme<TConditionalSession> {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TConditionalSession struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // SESSION_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.sessionId = iprot.readI64();
+ struct.setSessionIdIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // TSERVER_LOCK
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.tserverLock = iprot.readString();
+ struct.setTserverLockIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // TTL
+ if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+ struct.ttl = iprot.readI64();
+ struct.setTtlIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot, TConditionalSession struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ oprot.writeFieldBegin(SESSION_ID_FIELD_DESC);
+ oprot.writeI64(struct.sessionId);
+ oprot.writeFieldEnd();
+ if (struct.tserverLock != null) {
+ oprot.writeFieldBegin(TSERVER_LOCK_FIELD_DESC);
+ oprot.writeString(struct.tserverLock);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldBegin(TTL_FIELD_DESC);
+ oprot.writeI64(struct.ttl);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TConditionalSessionTupleSchemeFactory implements SchemeFactory {
+ public TConditionalSessionTupleScheme getScheme() {
+ return new TConditionalSessionTupleScheme();
+ }
+ }
+
+ private static class TConditionalSessionTupleScheme extends TupleScheme<TConditionalSession> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TConditionalSession struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetSessionId()) {
+ optionals.set(0);
+ }
+ if (struct.isSetTserverLock()) {
+ optionals.set(1);
+ }
+ if (struct.isSetTtl()) {
+ optionals.set(2);
+ }
+ oprot.writeBitSet(optionals, 3);
+ if (struct.isSetSessionId()) {
+ oprot.writeI64(struct.sessionId);
+ }
+ if (struct.isSetTserverLock()) {
+ oprot.writeString(struct.tserverLock);
+ }
+ if (struct.isSetTtl()) {
+ oprot.writeI64(struct.ttl);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TConditionalSession struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(3);
+ if (incoming.get(0)) {
+ struct.sessionId = iprot.readI64();
+ struct.setSessionIdIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.tserverLock = iprot.readString();
+ struct.setTserverLockIsSet(true);
+ }
+ if (incoming.get(2)) {
+ struct.ttl = iprot.readI64();
+ struct.setTtlIsSet(true);
+ }
+ }
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
index 4ba0eb6..97001ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
@@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.util.UnsynchronizedBuffer;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
@@ -472,37 +473,9 @@ public class RelativeKey implements Writable {
read(in, mbseq, len);
}
- /**
- * Determines what next array size should be by rounding up to next power of two.
- *
- */
- static int nextArraySize(int i) {
- if (i < 0)
- throw new IllegalArgumentException();
-
- if (i > (1 << 30))
- return Integer.MAX_VALUE; // this is the next power of 2 minus one... a special case
-
- if (i == 0) {
- return 1;
- }
-
- // round up to next power of two
- int ret = i;
- ret--;
- ret |= ret >> 1;
- ret |= ret >> 2;
- ret |= ret >> 4;
- ret |= ret >> 8;
- ret |= ret >> 16;
- ret++;
-
- return ret;
- }
-
private static void read(DataInput in, MByteSequence mbseqDestination, int len) throws IOException {
if (mbseqDestination.getBackingArray().length < len) {
- mbseqDestination.setArray(new byte[nextArraySize(len)]);
+ mbseqDestination.setArray(new byte[UnsynchronizedBuffer.nextArraySize(len)]);
}
in.readFully(mbseqDestination.getBackingArray(), 0, len);
@@ -529,7 +502,7 @@ public class RelativeKey implements Writable {
int remainingLen = WritableUtils.readVInt(in);
int len = prefixLen + remainingLen;
if (dest.getBackingArray().length < len) {
- dest.setArray(new byte[nextArraySize(len)]);
+ dest.setArray(new byte[UnsynchronizedBuffer.nextArraySize(len)]);
}
if (prefixSource.isBackedByArray()) {
System.arraycopy(prefixSource.getBackingArray(), prefixSource.offset(), dest.getBackingArray(), 0, prefixLen);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
index 1595f5a..d5ca3b4 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnQualifierFilter.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.iterators.system;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.Set;
import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ByteSequence;
@@ -36,7 +37,7 @@ public class ColumnQualifierFilter extends Filter {
public ColumnQualifierFilter() {}
- public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, HashSet<Column> columns) {
+ public ColumnQualifierFilter(SortedKeyValueIterator<Key,Value> iterator, Set<Column> columns) {
setSource(iterator);
init(columns);
}
@@ -63,7 +64,7 @@ public class ColumnQualifierFilter extends Filter {
return cfset != null && cfset.contains(key.getColumnFamilyData());
}
- public void init(HashSet<Column> columns) {
+ public void init(Set<Column> columns) {
this.columnFamilies = new HashSet<ByteSequence>();
this.columnsQualifiers = new HashMap<ByteSequence,HashSet<ByteSequence>>();
[3/6] ACCUMULO-1000 added conditional mutations to Accumulo
Posted by kt...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/9dc24448/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index 9144eb0..19fbb98 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -1,19 +1,3 @@
-/*
- * 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.
- */
/**
* Autogenerated by Thrift Compiler (0.9.0)
*
@@ -46,7 +30,7 @@ import java.util.Arrays;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-@SuppressWarnings("all") public class TabletClientService {
+public class TabletClientService {
public interface Iface extends org.apache.accumulo.core.client.impl.thrift.ClientService.Iface {
@@ -70,6 +54,14 @@ import org.slf4j.LoggerFactory;
public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException;
+ public org.apache.accumulo.core.data.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+ public List<org.apache.accumulo.core.data.thrift.TCMResult> conditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols) throws NoSuchScanIDException, org.apache.thrift.TException;
+
+ public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
+ public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
+
public List<org.apache.accumulo.core.data.thrift.TKeyExtent> bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
public void splitTablet(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, ByteBuffer splitPoint) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, NotServingTabletException, org.apache.thrift.TException;
@@ -126,6 +118,14 @@ import org.slf4j.LoggerFactory;
public void update(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.data.thrift.TMutation mutation, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.update_call> resultHandler) throws org.apache.thrift.TException;
+ public void startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.startConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException;
+
+ public void conditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.conditionalUpdate_call> resultHandler) throws org.apache.thrift.TException;
+
+ public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.invalidateConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException;
+
+ public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.closeConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException;
+
public void bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.bulkImport_call> resultHandler) throws org.apache.thrift.TException;
public void splitTablet(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.accumulo.core.data.thrift.TKeyExtent extent, ByteBuffer splitPoint, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.splitTablet_call> resultHandler) throws org.apache.thrift.TException;
@@ -453,6 +453,98 @@ import org.slf4j.LoggerFactory;
return;
}
+ public org.apache.accumulo.core.data.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ {
+ send_startConditionalUpdate(tinfo, credentials, authorizations, tableID);
+ return recv_startConditionalUpdate();
+ }
+
+ public void send_startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID) throws org.apache.thrift.TException
+ {
+ startConditionalUpdate_args args = new startConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setAuthorizations(authorizations);
+ args.setTableID(tableID);
+ sendBase("startConditionalUpdate", args);
+ }
+
+ public org.apache.accumulo.core.data.thrift.TConditionalSession recv_startConditionalUpdate() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+ {
+ startConditionalUpdate_result result = new startConditionalUpdate_result();
+ receiveBase(result, "startConditionalUpdate");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ if (result.sec != null) {
+ throw result.sec;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "startConditionalUpdate failed: unknown result");
+ }
+
+ public List<org.apache.accumulo.core.data.thrift.TCMResult> conditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols) throws NoSuchScanIDException, org.apache.thrift.TException
+ {
+ send_conditionalUpdate(tinfo, sessID, mutations, symbols);
+ return recv_conditionalUpdate();
+ }
+
+ public void send_conditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols) throws org.apache.thrift.TException
+ {
+ conditionalUpdate_args args = new conditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.setMutations(mutations);
+ args.setSymbols(symbols);
+ sendBase("conditionalUpdate", args);
+ }
+
+ public List<org.apache.accumulo.core.data.thrift.TCMResult> recv_conditionalUpdate() throws NoSuchScanIDException, org.apache.thrift.TException
+ {
+ conditionalUpdate_result result = new conditionalUpdate_result();
+ receiveBase(result, "conditionalUpdate");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ if (result.nssi != null) {
+ throw result.nssi;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "conditionalUpdate failed: unknown result");
+ }
+
+ public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ send_invalidateConditionalUpdate(tinfo, sessID);
+ recv_invalidateConditionalUpdate();
+ }
+
+ public void send_invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ sendBase("invalidateConditionalUpdate", args);
+ }
+
+ public void recv_invalidateConditionalUpdate() throws org.apache.thrift.TException
+ {
+ invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+ receiveBase(result, "invalidateConditionalUpdate");
+ return;
+ }
+
+ public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ send_closeConditionalUpdate(tinfo, sessID);
+ }
+
+ public void send_closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
+ {
+ closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ sendBase("closeConditionalUpdate", args);
+ }
+
public List<org.apache.accumulo.core.data.thrift.TKeyExtent> bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
{
send_bulkImport(tinfo, credentials, tid, files, setTime);
@@ -1219,6 +1311,157 @@ import org.slf4j.LoggerFactory;
}
}
+ public void startConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, org.apache.thrift.async.AsyncMethodCallback<startConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ startConditionalUpdate_call method_call = new startConditionalUpdate_call(tinfo, credentials, authorizations, tableID, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class startConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private org.apache.accumulo.trace.thrift.TInfo tinfo;
+ private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+ private List<ByteBuffer> authorizations;
+ private String tableID;
+ public startConditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<ByteBuffer> authorizations, String tableID, org.apache.thrift.async.AsyncMethodCallback<startConditionalUpdate_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.authorizations = authorizations;
+ this.tableID = tableID;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ startConditionalUpdate_args args = new startConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setCredentials(credentials);
+ args.setAuthorizations(authorizations);
+ args.setTableID(tableID);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public org.apache.accumulo.core.data.thrift.TConditionalSession getResult() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_startConditionalUpdate();
+ }
+ }
+
+ public void conditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols, org.apache.thrift.async.AsyncMethodCallback<conditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ conditionalUpdate_call method_call = new conditionalUpdate_call(tinfo, sessID, mutations, symbols, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class conditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private org.apache.accumulo.trace.thrift.TInfo tinfo;
+ private long sessID;
+ private Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations;
+ private List<String> symbols;
+ public conditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations, List<String> symbols, org.apache.thrift.async.AsyncMethodCallback<conditionalUpdate_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ this.mutations = mutations;
+ this.symbols = symbols;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("conditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ conditionalUpdate_args args = new conditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.setMutations(mutations);
+ args.setSymbols(symbols);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public List<org.apache.accumulo.core.data.thrift.TCMResult> getResult() throws NoSuchScanIDException, org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_conditionalUpdate();
+ }
+ }
+
+ public void invalidateConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<invalidateConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ invalidateConditionalUpdate_call method_call = new invalidateConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class invalidateConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private org.apache.accumulo.trace.thrift.TInfo tinfo;
+ private long sessID;
+ public invalidateConditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<invalidateConditionalUpdate_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("invalidateConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ invalidateConditionalUpdate_args args = new invalidateConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ (new Client(prot)).recv_invalidateConditionalUpdate();
+ }
+ }
+
+ public void closeConditionalUpdate(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<closeConditionalUpdate_call> resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ closeConditionalUpdate_call method_call = new closeConditionalUpdate_call(tinfo, sessID, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class closeConditionalUpdate_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private org.apache.accumulo.trace.thrift.TInfo tinfo;
+ private long sessID;
+ public closeConditionalUpdate_call(org.apache.accumulo.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<closeConditionalUpdate_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, true);
+ this.tinfo = tinfo;
+ this.sessID = sessID;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("closeConditionalUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ closeConditionalUpdate_args args = new closeConditionalUpdate_args();
+ args.setTinfo(tinfo);
+ args.setSessID(sessID);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public void getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ }
+ }
+
public void bulkImport(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long tid, Map<org.apache.accumulo.core.data.thrift.TKeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<bulkImport_call> resultHandler) throws org.apache.thrift.TException {
checkReady();
bulkImport_call method_call = new bulkImport_call(tinfo, credentials, tid, files, setTime, resultHandler, this, ___protocolFactory, ___transport);
@@ -1872,6 +2115,10 @@ import org.slf4j.LoggerFactory;
processMap.put("applyUpdates", new applyUpdates());
processMap.put("closeUpdate", new closeUpdate());
processMap.put("update", new update());
+ processMap.put("startConditionalUpdate", new startConditionalUpdate());
+ processMap.put("conditionalUpdate", new conditionalUpdate());
+ processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
+ processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
processMap.put("bulkImport", new bulkImport());
processMap.put("splitTablet", new splitTablet());
processMap.put("loadTablet", new loadTablet());
@@ -2134,6 +2381,93 @@ import org.slf4j.LoggerFactory;
}
}
+ public static class startConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startConditionalUpdate_args> {
+ public startConditionalUpdate() {
+ super("startConditionalUpdate");
+ }
+
+ public startConditionalUpdate_args getEmptyArgsInstance() {
+ return new startConditionalUpdate_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public startConditionalUpdate_result getResult(I iface, startConditionalUpdate_args args) throws org.apache.thrift.TException {
+ startConditionalUpdate_result result = new startConditionalUpdate_result();
+ try {
+ result.success = iface.startConditionalUpdate(args.tinfo, args.credentials, args.authorizations, args.tableID);
+ } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+ result.sec = sec;
+ }
+ return result;
+ }
+ }
+
+ public static class conditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, conditionalUpdate_args> {
+ public conditionalUpdate() {
+ super("conditionalUpdate");
+ }
+
+ public conditionalUpdate_args getEmptyArgsInstance() {
+ return new conditionalUpdate_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public conditionalUpdate_result getResult(I iface, conditionalUpdate_args args) throws org.apache.thrift.TException {
+ conditionalUpdate_result result = new conditionalUpdate_result();
+ try {
+ result.success = iface.conditionalUpdate(args.tinfo, args.sessID, args.mutations, args.symbols);
+ } catch (NoSuchScanIDException nssi) {
+ result.nssi = nssi;
+ }
+ return result;
+ }
+ }
+
+ public static class invalidateConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, invalidateConditionalUpdate_args> {
+ public invalidateConditionalUpdate() {
+ super("invalidateConditionalUpdate");
+ }
+
+ public invalidateConditionalUpdate_args getEmptyArgsInstance() {
+ return new invalidateConditionalUpdate_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public invalidateConditionalUpdate_result getResult(I iface, invalidateConditionalUpdate_args args) throws org.apache.thrift.TException {
+ invalidateConditionalUpdate_result result = new invalidateConditionalUpdate_result();
+ iface.invalidateConditionalUpdate(args.tinfo, args.sessID);
+ return result;
+ }
+ }
+
+ public static class closeConditionalUpdate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, closeConditionalUpdate_args> {
+ public closeConditionalUpdate() {
+ super("closeConditionalUpdate");
+ }
+
+ public closeConditionalUpdate_args getEmptyArgsInstance() {
+ return new closeConditionalUpdate_args();
+ }
+
+ protected boolean isOneway() {
+ return true;
+ }
+
+ public org.apache.thrift.TBase getResult(I iface, closeConditionalUpdate_args args) throws org.apache.thrift.TException {
+ iface.closeConditionalUpdate(args.tinfo, args.sessID);
+ return null;
+ }
+ }
+
public static class bulkImport<I extends Iface> extends org.apache.thrift.ProcessFunction<I, bulkImport_args> {
public bulkImport() {
super("bulkImport");
@@ -2516,7 +2850,7 @@ import org.slf4j.LoggerFactory;
public boolean isolated; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)11, "tinfo"),
CREDENTIALS((short)1, "credentials"),
EXTENT((short)2, "extent"),
@@ -3685,7 +4019,7 @@ import org.slf4j.LoggerFactory;
struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list106.size);
for (int _i107 = 0; _i107 < _list106.size; ++_i107)
{
- org.apache.accumulo.core.data.thrift.TColumn _elem108; // optional
+ org.apache.accumulo.core.data.thrift.TColumn _elem108; // required
_elem108 = new org.apache.accumulo.core.data.thrift.TColumn();
_elem108.read(iprot);
struct.columns.add(_elem108);
@@ -3712,7 +4046,7 @@ import org.slf4j.LoggerFactory;
struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list109.size);
for (int _i110 = 0; _i110 < _list109.size; ++_i110)
{
- org.apache.accumulo.core.data.thrift.IterInfo _elem111; // optional
+ org.apache.accumulo.core.data.thrift.IterInfo _elem111; // required
_elem111 = new org.apache.accumulo.core.data.thrift.IterInfo();
_elem111.read(iprot);
struct.ssiList.add(_elem111);
@@ -3763,7 +4097,7 @@ import org.slf4j.LoggerFactory;
struct.authorizations = new ArrayList<ByteBuffer>(_list120.size);
for (int _i121 = 0; _i121 < _list120.size; ++_i121)
{
- ByteBuffer _elem122; // optional
+ ByteBuffer _elem122; // required
_elem122 = iprot.readBinary();
struct.authorizations.add(_elem122);
}
@@ -4040,7 +4374,7 @@ import org.slf4j.LoggerFactory;
struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list133.size);
for (int _i134 = 0; _i134 < _list133.size; ++_i134)
{
- org.apache.accumulo.core.data.thrift.TColumn _elem135; // optional
+ org.apache.accumulo.core.data.thrift.TColumn _elem135; // required
_elem135 = new org.apache.accumulo.core.data.thrift.TColumn();
_elem135.read(iprot);
struct.columns.add(_elem135);
@@ -4058,7 +4392,7 @@ import org.slf4j.LoggerFactory;
struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list136.size);
for (int _i137 = 0; _i137 < _list136.size; ++_i137)
{
- org.apache.accumulo.core.data.thrift.IterInfo _elem138; // optional
+ org.apache.accumulo.core.data.thrift.IterInfo _elem138; // required
_elem138 = new org.apache.accumulo.core.data.thrift.IterInfo();
_elem138.read(iprot);
struct.ssiList.add(_elem138);
@@ -4098,7 +4432,7 @@ import org.slf4j.LoggerFactory;
struct.authorizations = new ArrayList<ByteBuffer>(_list147.size);
for (int _i148 = 0; _i148 < _list147.size; ++_i148)
{
- ByteBuffer _elem149; // optional
+ ByteBuffer _elem149; // required
_elem149 = iprot.readBinary();
struct.authorizations.add(_elem149);
}
@@ -4138,7 +4472,7 @@ import org.slf4j.LoggerFactory;
public TooManyFilesException tmfe; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
SUCCESS((short)0, "success"),
SEC((short)1, "sec"),
NSTE((short)2, "nste"),
@@ -4799,7 +5133,7 @@ import org.slf4j.LoggerFactory;
public long scanID; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)2, "tinfo"),
SCAN_ID((short)1, "scanID");
@@ -5260,7 +5594,7 @@ import org.slf4j.LoggerFactory;
public TooManyFilesException tmfe; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
SUCCESS((short)0, "success"),
NSSI((short)1, "nssi"),
NSTE((short)2, "nste"),
@@ -5921,7 +6255,7 @@ import org.slf4j.LoggerFactory;
public long scanID; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)2, "tinfo"),
SCAN_ID((short)1, "scanID");
@@ -6390,7 +6724,7 @@ import org.slf4j.LoggerFactory;
public boolean waitForWrites; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)8, "tinfo"),
CREDENTIALS((short)1, "credentials"),
BATCH((short)2, "batch"),
@@ -7338,7 +7672,7 @@ import org.slf4j.LoggerFactory;
_val153 = new ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list154.size);
for (int _i155 = 0; _i155 < _list154.size; ++_i155)
{
- org.apache.accumulo.core.data.thrift.TRange _elem156; // optional
+ org.apache.accumulo.core.data.thrift.TRange _elem156; // required
_elem156 = new org.apache.accumulo.core.data.thrift.TRange();
_elem156.read(iprot);
_val153.add(_elem156);
@@ -7361,7 +7695,7 @@ import org.slf4j.LoggerFactory;
struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list157.size);
for (int _i158 = 0; _i158 < _list157.size; ++_i158)
{
- org.apache.accumulo.core.data.thrift.TColumn _elem159; // optional
+ org.apache.accumulo.core.data.thrift.TColumn _elem159; // required
_elem159 = new org.apache.accumulo.core.data.thrift.TColumn();
_elem159.read(iprot);
struct.columns.add(_elem159);
@@ -7380,7 +7714,7 @@ import org.slf4j.LoggerFactory;
struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list160.size);
for (int _i161 = 0; _i161 < _list160.size; ++_i161)
{
- org.apache.accumulo.core.data.thrift.IterInfo _elem162; // optional
+ org.apache.accumulo.core.data.thrift.IterInfo _elem162; // required
_elem162 = new org.apache.accumulo.core.data.thrift.IterInfo();
_elem162.read(iprot);
struct.ssiList.add(_elem162);
@@ -7431,7 +7765,7 @@ import org.slf4j.LoggerFactory;
struct.authorizations = new ArrayList<ByteBuffer>(_list171.size);
for (int _i172 = 0; _i172 < _list171.size; ++_i172)
{
- ByteBuffer _elem173; // optional
+ ByteBuffer _elem173; // required
_elem173 = iprot.readBinary();
struct.authorizations.add(_elem173);
}
@@ -7698,7 +8032,7 @@ import org.slf4j.LoggerFactory;
_val191 = new ArrayList<org.apache.accumulo.core.data.thrift.TRange>(_list192.size);
for (int _i193 = 0; _i193 < _list192.size; ++_i193)
{
- org.apache.accumulo.core.data.thrift.TRange _elem194; // optional
+ org.apache.accumulo.core.data.thrift.TRange _elem194; // required
_elem194 = new org.apache.accumulo.core.data.thrift.TRange();
_elem194.read(iprot);
_val191.add(_elem194);
@@ -7715,7 +8049,7 @@ import org.slf4j.LoggerFactory;
struct.columns = new ArrayList<org.apache.accumulo.core.data.thrift.TColumn>(_list195.size);
for (int _i196 = 0; _i196 < _list195.size; ++_i196)
{
- org.apache.accumulo.core.data.thrift.TColumn _elem197; // optional
+ org.apache.accumulo.core.data.thrift.TColumn _elem197; // required
_elem197 = new org.apache.accumulo.core.data.thrift.TColumn();
_elem197.read(iprot);
struct.columns.add(_elem197);
@@ -7729,7 +8063,7 @@ import org.slf4j.LoggerFactory;
struct.ssiList = new ArrayList<org.apache.accumulo.core.data.thrift.IterInfo>(_list198.size);
for (int _i199 = 0; _i199 < _list198.size; ++_i199)
{
- org.apache.accumulo.core.data.thrift.IterInfo _elem200; // optional
+ org.apache.accumulo.core.data.thrift.IterInfo _elem200; // required
_elem200 = new org.apache.accumulo.core.data.thrift.IterInfo();
_elem200.read(iprot);
struct.ssiList.add(_elem200);
@@ -7769,7 +8103,7 @@ import org.slf4j.LoggerFactory;
struct.authorizations = new ArrayList<ByteBuffer>(_list209.size);
for (int _i210 = 0; _i210 < _list209.size; ++_i210)
{
- ByteBuffer _elem211; // optional
+ ByteBuffer _elem211; // required
_elem211 = iprot.readBinary();
struct.authorizations.add(_elem211);
}
@@ -7801,7 +8135,7 @@ import org.slf4j.LoggerFactory;
public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
SUCCESS((short)0, "success"),
SEC((short)1, "sec");
@@ -8262,7 +8596,7 @@ import org.slf4j.LoggerFactory;
public long scanID; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)2, "tinfo"),
SCAN_ID((short)1, "scanID");
@@ -8719,7 +9053,7 @@ import org.slf4j.LoggerFactory;
public NoSuchScanIDException nssi; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
SUCCESS((short)0, "success"),
NSSI((short)1, "nssi");
@@ -9180,7 +9514,7 @@ import org.slf4j.LoggerFactory;
public long scanID; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)2, "tinfo"),
SCAN_ID((short)1, "scanID");
@@ -9635,7 +9969,7 @@ import org.slf4j.LoggerFactory;
public NoSuchScanIDException nssi; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
NSSI((short)1, "nssi");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -9993,7 +10327,7 @@ import org.slf4j.LoggerFactory;
public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)2, "tinfo"),
CREDENTIALS((short)1, "credentials");
@@ -10457,7 +10791,7 @@ import org.slf4j.LoggerFactory;
public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
SUCCESS((short)0, "success"),
SEC((short)1, "sec");
@@ -10917,7 +11251,7 @@ import org.slf4j.LoggerFactory;
public List<org.apache.accumulo.core.data.thrift.TMutation> mutations; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)1, "tinfo"),
UPDATE_ID((short)2, "updateID"),
KEY_EXTENT((short)3, "keyExtent"),
@@ -11473,7 +11807,7 @@ import org.slf4j.LoggerFactory;
struct.mutations = new ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list212.size);
for (int _i213 = 0; _i213 < _list212.size; ++_i213)
{
- org.apache.accumulo.core.data.thrift.TMutation _elem214; // optional
+ org.apache.accumulo.core.data.thrift.TMutation _elem214; // required
_elem214 = new org.apache.accumulo.core.data.thrift.TMutation();
_elem214.read(iprot);
struct.mutations.add(_elem214);
@@ -11600,7 +11934,7 @@ import org.slf4j.LoggerFactory;
struct.mutations = new ArrayList<org.apache.accumulo.core.data.thrift.TMutation>(_list217.size);
for (int _i218 = 0; _i218 < _list217.size; ++_i218)
{
- org.apache.accumulo.core.data.thrift.TMutation _elem219; // optional
+ org.apache.accumulo.core.data.thrift.TMutation _elem219; // required
_elem219 = new org.apache.accumulo.core.data.thrift.TMutation();
_elem219.read(iprot);
struct.mutations.add(_elem219);
@@ -11629,7 +11963,7 @@ import org.slf4j.LoggerFactory;
public long updateID; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)2, "tinfo"),
UPDATE_ID((short)1, "updateID");
@@ -12086,7 +12420,7 @@ import org.slf4j.LoggerFactory;
public NoSuchScanIDException nssi; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
SUCCESS((short)0, "success"),
NSSI((short)1, "nssi");
@@ -12551,7 +12885,7 @@ import org.slf4j.LoggerFactory;
public org.apache.accumulo.core.data.thrift.TMutation mutation; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
TINFO((short)4, "tinfo"),
CREDENTIALS((short)1, "credentials"),
KEY_EXTENT((short)2, "keyExtent"),
@@ -13223,7 +13557,7 @@ import org.slf4j.LoggerFactory;
public ConstraintViolationException cve; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
- @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
SEC((short)1, "sec"),
NSTE((short)2, "nste"),
CVE((short)3, "cve");
@@ -13765,6 +14099,3649 @@ import org.slf4j.LoggerFactory;
}
+ public static class startConditionalUpdate_args implements org.apache.thrift.TBase<startConditionalUpdate_args, startConditionalUpdate_args._Fields>, java.io.Serializable, Cloneable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+ private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.LIST, (short)3);
+ private static final org.apache.thrift.protocol.TField TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableID", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+ private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new startConditionalUpdate_argsStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new startConditionalUpdate_argsTupleSchemeFactory());
+ }
+
+ public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+ public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+ public List<ByteBuffer> authorizations; // required
+ public String tableID; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ CREDENTIALS((short)2, "credentials"),
+ AUTHORIZATIONS((short)3, "authorizations"),
+ TABLE_ID((short)4, "tableID");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // TINFO
+ return TINFO;
+ case 2: // CREDENTIALS
+ return CREDENTIALS;
+ case 3: // AUTHORIZATIONS
+ return AUTHORIZATIONS;
+ case 4: // TABLE_ID
+ return TABLE_ID;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
+ tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+ tmpMap.put(_Fields.AUTHORIZATIONS, new org.apache.thrift.meta_data.FieldMetaData("authorizations", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
+ tmpMap.put(_Fields.TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("tableID", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startConditionalUpdate_args.class, metaDataMap);
+ }
+
+ public startConditionalUpdate_args() {
+ }
+
+ public startConditionalUpdate_args(
+ org.apache.accumulo.trace.thrift.TInfo tinfo,
+ org.apache.accumulo.core.security.thrift.TCredentials credentials,
+ List<ByteBuffer> authorizations,
+ String tableID)
+ {
+ this();
+ this.tinfo = tinfo;
+ this.credentials = credentials;
+ this.authorizations = authorizations;
+ this.tableID = tableID;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public startConditionalUpdate_args(startConditionalUpdate_args other) {
+ if (other.isSetTinfo()) {
+ this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
+ }
+ if (other.isSetCredentials()) {
+ this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+ }
+ if (other.isSetAuthorizations()) {
+ List<ByteBuffer> __this__authorizations = new ArrayList<ByteBuffer>();
+ for (ByteBuffer other_element : other.authorizations) {
+ ByteBuffer temp_binary_element = org.apache.thrift.TBaseHelper.copyBinary(other_element);
+;
+ __this__authorizations.add(temp_binary_element);
+ }
+ this.authorizations = __this__authorizations;
+ }
+ if (other.isSetTableID()) {
+ this.tableID = other.tableID;
+ }
+ }
+
+ public startConditionalUpdate_args deepCopy() {
+ return new startConditionalUpdate_args(this);
+ }
+
+ @Override
+ public void clear() {
+ this.tinfo = null;
+ this.credentials = null;
+ this.authorizations = null;
+ this.tableID = null;
+ }
+
+ public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
+ return this.tinfo;
+ }
+
+ public startConditionalUpdate_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+ this.tinfo = tinfo;
+ return this;
+ }
+
+ public void unsetTinfo() {
+ this.tinfo = null;
+ }
+
+ /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+ public boolean isSetTinfo() {
+ return this.tinfo != null;
+ }
+
+ public void setTinfoIsSet(boolean value) {
+ if (!value) {
+ this.tinfo = null;
+ }
+ }
+
+ public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+ return this.credentials;
+ }
+
+ public startConditionalUpdate_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public void unsetCredentials() {
+ this.credentials = null;
+ }
+
+ /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+ public boolean isSetCredentials() {
+ return this.credentials != null;
+ }
+
+ public void setCredentialsIsSet(boolean value) {
+ if (!value) {
+ this.credentials = null;
+ }
+ }
+
+ public int getAuthorizationsSize() {
+ return (this.authorizations == null) ? 0 : this.authorizations.size();
+ }
+
+ public java.util.Iterator<ByteBuffer> getAuthorizationsIterator() {
+ return (this.authorizations == null) ? null : this.authorizations.iterator();
+ }
+
+ public void addToAuthorizations(ByteBuffer elem) {
+ if (this.authorizations == null) {
+ this.authorizations = new ArrayList<ByteBuffer>();
+ }
+ this.authorizations.add(elem);
+ }
+
+ public List<ByteBuffer> getAuthorizations() {
+ return this.authorizations;
+ }
+
+ public startConditionalUpdate_args setAuthorizations(List<ByteBuffer> authorizations) {
+ this.authorizations = authorizations;
+ return this;
+ }
+
+ public void unsetAuthorizations() {
+ this.authorizations = null;
+ }
+
+ /** Returns true if field authorizations is set (has been assigned a value) and false otherwise */
+ public boolean isSetAuthorizations() {
+ return this.authorizations != null;
+ }
+
+ public void setAuthorizationsIsSet(boolean value) {
+ if (!value) {
+ this.authorizations = null;
+ }
+ }
+
+ public String getTableID() {
+ return this.tableID;
+ }
+
+ public startConditionalUpdate_args setTableID(String tableID) {
+ this.tableID = tableID;
+ return this;
+ }
+
+ public void unsetTableID() {
+ this.tableID = null;
+ }
+
+ /** Returns true if field tableID is set (has been assigned a value) and false otherwise */
+ public boolean isSetTableID() {
+ return this.tableID != null;
+ }
+
+ public void setTableIDIsSet(boolean value) {
+ if (!value) {
+ this.tableID = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case TINFO:
+ if (value == null) {
+ unsetTinfo();
+ } else {
+ setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
+ }
+ break;
+
+ case CREDENTIALS:
+ if (value == null) {
+ unsetCredentials();
+ } else {
+ setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+ }
+ break;
+
+ case AUTHORIZATIONS:
+ if (value == null) {
+ unsetAuthorizations();
+ } else {
+ setAuthorizations((List<ByteBuffer>)value);
+ }
+ break;
+
+ case TABLE_ID:
+ if (value == null) {
+ unsetTableID();
+ } else {
+ setTableID((String)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case TINFO:
+ return getTinfo();
+
+ case CREDENTIALS:
+ return getCredentials();
+
+ case AUTHORIZATIONS:
+ return getAuthorizations();
+
+ case TABLE_ID:
+ return getTableID();
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case TINFO:
+ return isSetTinfo();
+ case CREDENTIALS:
+ return isSetCredentials();
+ case AUTHORIZATIONS:
+ return isSetAuthorizations();
+ case TABLE_ID:
+ return isSetTableID();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof startConditionalUpdate_args)
+ return this.equals((startConditionalUpdate_args)that);
+ return false;
+ }
+
+ public boolean equals(startConditionalUpdate_args that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_tinfo = true && this.isSetTinfo();
+ boolean that_present_tinfo = true && that.isSetTinfo();
+ if (this_present_tinfo || that_present_tinfo) {
+ if (!(this_present_tinfo && that_present_tinfo))
+ return false;
+ if (!this.tinfo.equals(that.tinfo))
+ return false;
+ }
+
+ boolean this_present_credentials = true && this.isSetCredentials();
+ boolean that_present_credentials = true && that.isSetCredentials();
+ if (this_present_credentials || that_present_credentials) {
+ if (!(this_present_credentials && that_present_credentials))
+ return false;
+ if (!this.credentials.equals(that.credentials))
+ return false;
+ }
+
+ boolean this_present_authorizations = true && this.isSetAuthorizations();
+ boolean that_present_authorizations = true && that.isSetAuthorizations();
+ if (this_present_authorizations || that_present_authorizations) {
+ if (!(this_present_authorizations && that_present_authorizations))
+ return false;
+ if (!this.authorizations.equals(that.authorizations))
+ return false;
+ }
+
+ boolean this_present_tableID = true && this.isSetTableID();
+ boolean that_present_tableID = true && that.isSetTableID();
+ if (this_present_tableID || that_present_tableID) {
+ if (!(this_present_tableID && that_present_tableID))
+ return false;
+ if (!this.tableID.equals(that.tableID))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(startConditionalUpdate_args other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ startConditionalUpdate_args typedOther = (startConditionalUpdate_args)other;
+
+ lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTinfo()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetCredentials()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetAuthorizations()).compareTo(typedOther.isSetAuthorizations());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetAuthorizations()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizations, typedOther.authorizations);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetTableID()).compareTo(typedOther.isSetTableID());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetTableID()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableID, typedOther.tableID);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("startConditionalUpdate_args(");
+ boolean first = true;
+
+ sb.append("tinfo:");
+ if (this.tinfo == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tinfo);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("credentials:");
+ if (this.credentials == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.credentials);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("authorizations:");
+ if (this.authorizations == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.authorizations);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("tableID:");
+ if (this.tableID == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.tableID);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (tinfo != null) {
+ tinfo.validate();
+ }
+ if (credentials != null) {
+ credentials.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class startConditionalUpdate_argsStandardSchemeFactory implements SchemeFactory {
+ public startConditionalUpdate_argsStandardScheme getScheme() {
+ return new startConditionalUpdate_argsStandardScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_argsStandardScheme extends StandardScheme<startConditionalUpdate_args> {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 1: // TINFO
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // CREDENTIALS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 3: // AUTHORIZATIONS
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list220 = iprot.readListBegin();
+ struct.authorizations = new ArrayList<ByteBuffer>(_list220.size);
+ for (int _i221 = 0; _i221 < _list220.size; ++_i221)
+ {
+ ByteBuffer _elem222; // required
+ _elem222 = iprot.readBinary();
+ struct.authorizations.add(_elem222);
+ }
+ iprot.readListEnd();
+ }
+ struct.setAuthorizationsIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 4: // TABLE_ID
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.tableID = iprot.readString();
+ struct.setTableIDIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.tinfo != null) {
+ oprot.writeFieldBegin(TINFO_FIELD_DESC);
+ struct.tinfo.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.credentials != null) {
+ oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+ struct.credentials.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.authorizations != null) {
+ oprot.writeFieldBegin(AUTHORIZATIONS_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.authorizations.size()));
+ for (ByteBuffer _iter223 : struct.authorizations)
+ {
+ oprot.writeBinary(_iter223);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.tableID != null) {
+ oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+ oprot.writeString(struct.tableID);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class startConditionalUpdate_argsTupleSchemeFactory implements SchemeFactory {
+ public startConditionalUpdate_argsTupleScheme getScheme() {
+ return new startConditionalUpdate_argsTupleScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_argsTupleScheme extends TupleScheme<startConditionalUpdate_args> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetTinfo()) {
+ optionals.set(0);
+ }
+ if (struct.isSetCredentials()) {
+ optionals.set(1);
+ }
+ if (struct.isSetAuthorizations()) {
+ optionals.set(2);
+ }
+ if (struct.isSetTableID()) {
+ optionals.set(3);
+ }
+ oprot.writeBitSet(optionals, 4);
+ if (struct.isSetTinfo()) {
+ struct.tinfo.write(oprot);
+ }
+ if (struct.isSetCredentials()) {
+ struct.credentials.write(oprot);
+ }
+ if (struct.isSetAuthorizations()) {
+ {
+ oprot.writeI32(struct.authorizations.size());
+ for (ByteBuffer _iter224 : struct.authorizations)
+ {
+ oprot.writeBinary(_iter224);
+ }
+ }
+ }
+ if (struct.isSetTableID()) {
+ oprot.writeString(struct.tableID);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_args struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(4);
+ if (incoming.get(0)) {
+ struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+ struct.tinfo.read(iprot);
+ struct.setTinfoIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+ struct.credentials.read(iprot);
+ struct.setCredentialsIsSet(true);
+ }
+ if (incoming.get(2)) {
+ {
+ org.apache.thrift.protocol.TList _list225 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+ struct.authorizations = new ArrayList<ByteBuffer>(_list225.size);
+ for (int _i226 = 0; _i226 < _list225.size; ++_i226)
+ {
+ ByteBuffer _elem227; // required
+ _elem227 = iprot.readBinary();
+ struct.authorizations.add(_elem227);
+ }
+ }
+ struct.setAuthorizationsIsSet(true);
+ }
+ if (incoming.get(3)) {
+ struct.tableID = iprot.readString();
+ struct.setTableIDIsSet(true);
+ }
+ }
+ }
+
+ }
+
+ public static class startConditionalUpdate_result implements org.apache.thrift.TBase<startConditionalUpdate_result, startConditionalUpdate_result._Fields>, java.io.Serializable, Cloneable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startConditionalUpdate_result");
+
+ private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+ private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+ private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new startConditionalUpdate_resultStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new startConditionalUpdate_resultTupleSchemeFactory());
+ }
+
+ public org.apache.accumulo.core.data.thrift.TConditionalSession success; // required
+ public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ SUCCESS((short)0, "success"),
+ SEC((short)1, "sec");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 0: // SUCCESS
+ return SUCCESS;
+ case 1: // SEC
+ return SEC;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.data.thrift.TConditionalSession.class)));
+ tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startConditionalUpdate_result.class, metaDataMap);
+ }
+
+ public startConditionalUpdate_result() {
+ }
+
+ public startConditionalUpdate_result(
+ org.apache.accumulo.core.data.thrift.TConditionalSession success,
+ org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec)
+ {
+ this();
+ this.success = success;
+ this.sec = sec;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public startConditionalUpdate_result(startConditionalUpdate_result other) {
+ if (other.isSetSuccess()) {
+ this.success = new org.apache.accumulo.core.data.thrift.TConditionalSession(other.success);
+ }
+ if (other.isSetSec()) {
+ this.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException(other.sec);
+ }
+ }
+
+ public startConditionalUpdate_result deepCopy() {
+ return new startConditionalUpdate_result(this);
+ }
+
+ @Override
+ public void clear() {
+ this.success = null;
+ this.sec = null;
+ }
+
+ public org.apache.accumulo.core.data.thrift.TConditionalSession getSuccess() {
+ return this.success;
+ }
+
+ public startConditionalUpdate_result setSuccess(org.apache.accumulo.core.data.thrift.TConditionalSession success) {
+ this.success = success;
+ return this;
+ }
+
+ public void unsetSuccess() {
+ this.success = null;
+ }
+
+ /** Returns true if field success is set (has been assigned a value) and false otherwise */
+ public boolean isSetSuccess() {
+ return this.success != null;
+ }
+
+ public void setSuccessIsSet(boolean value) {
+ if (!value) {
+ this.success = null;
+ }
+ }
+
+ public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException getSec() {
+ return this.sec;
+ }
+
+ public startConditionalUpdate_result setSec(org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+ this.sec = sec;
+ return this;
+ }
+
+ public void unsetSec() {
+ this.sec = null;
+ }
+
+ /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+ public boolean isSetSec() {
+ return this.sec != null;
+ }
+
+ public void setSecIsSet(boolean value) {
+ if (!value) {
+ this.sec = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case SUCCESS:
+ if (value == null) {
+ unsetSuccess();
+ } else {
+ setSuccess((org.apache.accumulo.core.data.thrift.TConditionalSession)value);
+ }
+ break;
+
+ case SEC:
+ if (value == null) {
+ unsetSec();
+ } else {
+ setSec((org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case SUCCESS:
+ return getSuccess();
+
+ case SEC:
+ return getSec();
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case SUCCESS:
+ return isSetSuccess();
+ case SEC:
+ return isSetSec();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof startConditionalUpdate_result)
+ return this.equals((startConditionalUpdate_result)that);
+ return false;
+ }
+
+ public boolean equals(startConditionalUpdate_result that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_success = true && this.isSetSuccess();
+ boolean that_present_success = true && that.isSetSuccess();
+ if (this_present_success || that_present_success) {
+ if (!(this_present_success && that_present_success))
+ return false;
+ if (!this.success.equals(that.success))
+ return false;
+ }
+
+ boolean this_present_sec = true && this.isSetSec();
+ boolean that_present_sec = true && that.isSetSec();
+ if (this_present_sec || that_present_sec) {
+ if (!(this_present_sec && that_present_sec))
+ return false;
+ if (!this.sec.equals(that.sec))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(startConditionalUpdate_result other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ startConditionalUpdate_result typedOther = (startConditionalUpdate_result)other;
+
+ lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSuccess()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetSec()).compareTo(typedOther.isSetSec());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetSec()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, typedOther.sec);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+ schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+ schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("startConditionalUpdate_result(");
+ boolean first = true;
+
+ sb.append("success:");
+ if (this.success == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.success);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("sec:");
+ if (this.sec == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.sec);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ // check for sub-struct validity
+ if (success != null) {
+ success.validate();
+ }
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class startConditionalUpdate_resultStandardSchemeFactory implements SchemeFactory {
+ public startConditionalUpdate_resultStandardScheme getScheme() {
+ return new startConditionalUpdate_resultStandardScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_resultStandardScheme extends StandardScheme<startConditionalUpdate_result> {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ org.apache.thrift.protocol.TField schemeField;
+ iprot.readStructBegin();
+ while (true)
+ {
+ schemeField = iprot.readFieldBegin();
+ if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+ break;
+ }
+ switch (schemeField.id) {
+ case 0: // SUCCESS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.success = new org.apache.accumulo.core.data.thrift.TConditionalSession();
+ struct.success.read(iprot);
+ struct.setSuccessIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 1: // SEC
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+ struct.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ default:
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ struct.validate();
+ }
+
+ public void write(org.apache.thrift.protocol.TProtocol oprot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.success != null) {
+ oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+ struct.success.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ if (struct.sec != null) {
+ oprot.writeFieldBegin(SEC_FIELD_DESC);
+ struct.sec.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class startConditionalUpdate_resultTupleSchemeFactory implements SchemeFactory {
+ public startConditionalUpdate_resultTupleScheme getScheme() {
+ return new startConditionalUpdate_resultTupleScheme();
+ }
+ }
+
+ private static class startConditionalUpdate_resultTupleScheme extends TupleScheme<startConditionalUpdate_result> {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetSuccess()) {
+ optionals.set(0);
+ }
+ if (struct.isSetSec()) {
+ optionals.set(1);
+ }
+ oprot.writeBitSet(optionals, 2);
+ if (struct.isSetSuccess()) {
+ struct.success.write(oprot);
+ }
+ if (struct.isSetSec()) {
+ struct.sec.write(oprot);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, startConditionalUpdate_result struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(2);
+ if (incoming.get(0)) {
+ struct.success = new org.apache.accumulo.core.data.thrift.TConditionalSession();
+ struct.success.read(iprot);
+ struct.setSuccessIsSet(true);
+ }
+ if (incoming.get(1)) {
+ struct.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException();
+ struct.sec.read(iprot);
+ struct.setSecIsSet(true);
+ }
+ }
+ }
+
+ }
+
+ public static class conditionalUpdate_args implements org.apache.thrift.TBase<conditionalUpdate_args, conditionalUpdate_args._Fields>, java.io.Serializable, Cloneable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("conditionalUpdate_args");
+
+ private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+ private static final org.apache.thrift.protocol.TField SESS_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessID", org.apache.thrift.protocol.TType.I64, (short)2);
+ private static final org.apache.thrift.protocol.TField MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("mutations", org.apache.thrift.protocol.TType.MAP, (short)3);
+ private static final org.apache.thrift.protocol.TField SYMBOLS_FIELD_DESC = new org.apache.thrift.protocol.TField("symbols", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+ private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new conditionalUpdate_argsStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new conditionalUpdate_argsTupleSchemeFactory());
+ }
+
+ public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+ public long sessID; // required
+ public Map<org.apache.accumulo.core.data.thrift.TKeyExtent,List<org.apache.accumulo.core.data.thrift.TConditionalMutation>> mutations; // required
+ public List<String> symbols; // required
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ TINFO((short)1, "tinfo"),
+ SESS_ID((short)2, "sessID"),
+ MUTATIONS((short)3, "mutations"),
+ SYMBOLS((short)4, "symbols");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
<TRUNCATED>