Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 40DC6CC5C for ; Wed, 17 Jul 2013 12:58:26 +0000 (UTC) Received: (qmail 70192 invoked by uid 500); 17 Jul 2013 12:58:20 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 70098 invoked by uid 500); 17 Jul 2013 12:58:20 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 69736 invoked by uid 99); 17 Jul 2013 12:58:15 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Jul 2013 12:58:15 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 2F7AC8ABC85; Wed, 17 Jul 2013 12:58:15 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: kturner@apache.org To: commits@accumulo.apache.org Date: Wed, 17 Jul 2013 12:58:18 -0000 Message-Id: In-Reply-To: <686a29cba05b4479ae57caedf5b65102@git.apache.org> References: <686a29cba05b4479ae57caedf5b65102@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [04/10] git commit: ACCUMULO-1000 initial checkin of conditional mutations that does locking on tablet server. The implementation is pretty far along, but still a good bit to do. ACCUMULO-1000 initial checkin of conditional mutations that does locking on tablet server. The implementation is pretty far along, but still a good bit to do. Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/49a7626c Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/49a7626c Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/49a7626c Branch: refs/heads/ACCUMULO-1000 Commit: 49a7626c359a5f25a2f8c103f4909d83c40ab4f0 Parents: 8d49260 Author: Keith Turner Authored: Mon Jul 15 13:36:28 2013 -0400 Committer: Keith Turner Committed: Mon Jul 15 13:36:28 2013 -0400 ---------------------------------------------------------------------- .../accumulo/core/client/ConditionalWriter.java | 97 ++ .../apache/accumulo/core/client/Connector.java | 16 + .../core/client/impl/ConditionalWriterImpl.java | 380 +++++ .../core/client/impl/ConnectorImpl.java | 9 +- .../client/impl/TabletServerBatchWriter.java | 2 + .../core/client/mock/MockConnector.java | 7 + .../apache/accumulo/core/data/Condition.java | 148 ++ .../accumulo/core/data/ConditionalMutation.java | 77 + .../accumulo/core/data/thrift/TCMResult.java | 516 ++++++ .../accumulo/core/data/thrift/TCMStatus.java | 67 + .../accumulo/core/data/thrift/TCondition.java | 1306 ++++++++++++++ .../core/data/thrift/TConditionalMutation.java | 659 +++++++ .../thrift/TabletClientService.java | 1603 ++++++++++++++++-- .../accumulo/core/util/ByteBufferUtil.java | 13 + core/src/main/thrift/data.thrift | 31 + core/src/main/thrift/tabletserver.thrift | 4 +- .../server/data/ServerConditionalMutation.java | 58 + .../tabletserver/ConditionalMutationSet.java | 91 + .../accumulo/server/tabletserver/RowLocks.java | 162 ++ .../server/tabletserver/TabletServer.java | 202 +++ .../accumulo/test/FaultyConditionalWriter.java | 85 + .../test/performance/thrift/NullTserver.java | 8 + .../accumulo/test/ConditionalWriterTest.java | 740 ++++++++ 23 files changed, 6129 insertions(+), 152 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/49a7626c/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..a23abcf --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/client/ConditionalWriter.java @@ -0,0 +1,97 @@ +/* + * 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 java.util.concurrent.TimeUnit; + +import org.apache.accumulo.core.data.ConditionalMutation; + +/** + * @since 1.6.0 + */ +public interface ConditionalWriter { + public static class Result { + + private Status status; + private ConditionalMutation mutation; + + public Result(Status s, ConditionalMutation m) { + this.status = s; + this.mutation = m; + } + + public Status getStatus() { + return status; + } + + public ConditionalMutation getMutation() { + return mutation; + } + } + + 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 + */ + UNKNOWN, + /** + * A condition contained a column visibility that could never be seen + */ + INVISIBLE_VISIBILITY, + /** + * nothing was done with this mutation, this is caused by previous mutations failing in some way like timing out + */ + IGNORED + } + + public abstract Iterator write(Iterator mutations); + + public abstract Result write(ConditionalMutation mutation); + + /** + * This setting determines how long a scanner will automatically retry when a failure occurs. By default a scanner will retry forever. + * + * Setting to zero or Long.MAX_VALUE and TimeUnit.MILLISECONDS means to retry forever. + * + * @param timeOut + * @param timeUnit + * determines how timeout is interpreted + */ + public void setTimeout(long timeOut, TimeUnit timeUnit); + + /** + * Returns the setting for how long a scanner will automatically retry when a failure occurs. + * + * @return the timeout configured for this scanner + */ + public long getTimeout(TimeUnit timeUnit); + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/49a7626c/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..45a8162 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,22 @@ 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 authorizations + * 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. + * + * @return ConditionalWriter object for writing ConditionalMutations + * @throws TableNotFoundException + * when the specified table doesn't exist + */ + public abstract ConditionalWriter createConditionalWriter(String tableName, Authorizations authorizations) throws TableNotFoundException; + + /** * Accessor method for internal instance object. * * @return the internal instance object http://git-wip-us.apache.org/repos/asf/accumulo/blob/49a7626c/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..73aa480 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java @@ -0,0 +1,380 @@ +/* + * 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.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.Set; +import java.util.concurrent.TimeUnit; + +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.Instance; +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations; +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.Mutation; +import org.apache.accumulo.core.data.thrift.IterInfo; +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.TKeyExtent; +import org.apache.accumulo.core.data.thrift.TMutation; +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.TabletClientService; +import org.apache.accumulo.core.util.BadArgumentException; +import org.apache.accumulo.core.util.ByteBufferUtil; +import org.apache.accumulo.core.util.ThriftUtil; +import org.apache.accumulo.core.util.UtilWaitThread; +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.thrift.TApplicationException; +import org.apache.thrift.TException; +import org.apache.thrift.TServiceClient; +import org.apache.thrift.transport.TTransportException; + + +class ConditionalWriterImpl implements ConditionalWriter { + + private Text tableId; + private Authorizations auths; + private VisibilityEvaluator ve; + private Map cache; + private Instance instance; + private TCredentials credentials; + + ConditionalWriterImpl(Instance instance, TCredentials credentials, String tableId, Authorizations authorizations) { + cache = Collections.synchronizedMap(new LRUMap(1000)); + this.instance = instance; + this.credentials = credentials; + this.tableId = new Text(tableId); + this.auths = authorizations; + this.ve = new VisibilityEvaluator(authorizations); + } + + public Iterator write(Iterator mutations) { + + + TabletLocator locator = TabletLocator.getLocator(instance, tableId); + + List mutationList = new ArrayList(); + + ArrayList results = new ArrayList(); + + mloop: while (mutations.hasNext()) { + ConditionalMutation mut = mutations.next(); + + for (Condition cond : mut.getConditions()) { + if (!isVisible(cond.getVisibility())) { + results.add(new Result(Status.INVISIBLE_VISIBILITY, mut)); + continue mloop; + } + } + + mutationList.add(mut); + } + + try { + List ignored = (List) (ArrayList) sendToServers(locator, mutationList, results); + + while (ignored.size() > 0) { + // TODO requeue ignored and return whats done for iteration + ignored = (List) (ArrayList) sendToServers(locator, ignored, results); + } + + return results.iterator(); + } catch (AccumuloException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } catch (AccumuloSecurityException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } catch (TableNotFoundException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + + return null; + } + + private class SendTask implements Runnable { + + private TabletServerMutations mutations; + private String location; + private ArrayList results; + private List ignored; + private TabletLocator locator; + + public SendTask(String location, TabletServerMutations mutations, ArrayList results, ArrayList ignored, TabletLocator locator) { + this.location = location; + this.mutations = mutations; + this.results = results; + this.ignored = ignored; + this.locator = locator; + } + + @Override + public void run() { + ArrayList tmpResults = new ArrayList(); + List tmpIgnored = new ArrayList(); + + sendToServer(location, mutations, tmpResults, tmpIgnored, locator); + + synchronized (results) { + results.addAll(tmpResults); + ignored.addAll(tmpIgnored); + } + } + } + protected ArrayList sendToServers(TabletLocator locator, List mutationList, ArrayList results) throws AccumuloException, + AccumuloSecurityException, TableNotFoundException { + + List failures = new ArrayList(); + Map binnedMutations = new HashMap(); + + do { + binnedMutations.clear(); + failures.clear(); + + locator.binMutations(mutationList, binnedMutations, failures, credentials); + + // TODO queue failed mutations to be retried in a bit and write what can be written + if (failures.size() > 0) + UtilWaitThread.sleep(100); + + } while (failures.size() > 0); + + ArrayList ignored = new ArrayList(); + + ArrayList threads = new ArrayList(); + + for (Entry entry : binnedMutations.entrySet()) { + Thread t = new Thread(new SendTask(entry.getKey(), entry.getValue(), results, ignored, locator)); + threads.add(t); + t.start(); + } + + for (Thread thread : threads) { + try { + thread.join(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + + return ignored; + } + + private static class CMK { + + ConditionalMutation cm; + KeyExtent ke; + + public CMK(KeyExtent ke, ConditionalMutation cm) { + this.ke = ke; + this.cm = cm; + } + } + + private void sendToServer(String location, TabletServerMutations mutations, ArrayList results, List ignored, + TabletLocator locator) { + TabletClientService.Iface client = null; + + TInfo tinfo = Tracer.traceInfo(); + + Map cmidToCm = new HashMap(); + MutableLong cmid = new MutableLong(0); + + try { + client = ThriftUtil.getTServerClient(location, instance.getConfiguration()); + + Map> tmutations = new HashMap>(); + + convertMutations(mutations, cmidToCm, cmid, tmutations); + + List tresults = client.conditionalUpdate(tinfo, credentials, ByteBufferUtil.toByteBuffers(auths.getAuthorizations()), tmutations); + + HashSet extentsToInvalidate = new HashSet(); + + for (TCMResult tcmResult : tresults) { + if (tcmResult.status == TCMStatus.IGNORED) { + CMK cmk = cmidToCm.get(tcmResult.cmid); + ignored.add(cmk.cm); + extentsToInvalidate.add(cmk.ke); + } else { + results.add(new Result(fromThrift(tcmResult.status), cmidToCm.get(tcmResult.cmid).cm)); + } + } + + // TODO maybe have thrift call return bad extents + + for (KeyExtent ke : extentsToInvalidate) { + locator.invalidateCache(ke); + } + + } catch (TTransportException e) { + locator.invalidateCache(location); + for (CMK cmk : cmidToCm.values()) + results.add(new Result(Status.UNKNOWN, cmk.cm)); + } catch (TApplicationException tae) { + for (CMK cmk : cmidToCm.values()) + results.add(new Result(Status.UNKNOWN, cmk.cm)); + // TODO should another status be used? + // TODO need to get server where error occurred back to client + } catch (TException e) { + locator.invalidateCache(location); + for (CMK cmk : cmidToCm.values()) + results.add(new Result(Status.UNKNOWN, cmk.cm)); + } 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 mutations, Map cmidToCm, MutableLong cmid, + Map> tmutations) { + + // TODO compress repeated iterator configurations + + Set>> es = mutations.getMutations().entrySet(); + for (Entry> entry : es) { + TKeyExtent tke = entry.getKey().toThrift(); + ArrayList tcondMutaions = new ArrayList(); + + List condMutations = (List) (List) entry.getValue(); + + for (ConditionalMutation cm : condMutations) { + TMutation tm = cm.toThrift(); + + + List conditions = convertConditions(cm); + + 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 convertConditions(ConditionalMutation cm) { + List conditions = new ArrayList(cm.getConditions().size()); + + for (Condition cond : cm.getConditions()) { + long ts = 0; + boolean hasTs = false; + + if (cond.getTimestamp() != null) { + ts = cond.getTimestamp(); + hasTs = true; + } + + IteratorSetting[] iters = cond.getIterators(); + + List ssiList = new ArrayList(iters.length); + Map> sso = new HashMap>(); + + if (iters.length == 0) { + ssiList = Collections.emptyList(); + sso = Collections.emptyMap(); + } else { + ssiList = new ArrayList(iters.length); + sso = new HashMap>(); + + for (IteratorSetting is : iters) { + ssiList.add(new IterInfo(is.getPriority(), is.getIteratorClass(), is.getName())); + sso.put(is.getName(), is.getOptions()); + } + } + + TCondition tc = new TCondition(ByteBufferUtil.toByteBuffers(cond.getFamily()), ByteBufferUtil.toByteBuffers(cond.getQualifier()), + ByteBufferUtil.toByteBuffers(cond.getVisibility()), ts, hasTs, ByteBufferUtil.toByteBuffers(cond.getValue()), ssiList, sso); + + conditions.add(tc); + } + + return conditions; + } + + private boolean isVisible(ByteSequence cv) { + Text testVis = new Text(cv.toArray()); + if (testVis.getLength() == 0) + return true; + + Boolean b = (Boolean) 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(); + } + + public void setTimeout(long timeOut, TimeUnit timeUnit) { + throw new UnsupportedOperationException(); + } + + public long getTimeout(TimeUnit timeUnit) { + throw new UnsupportedOperationException(); + } + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/49a7626c/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 1702082..9851ea0 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,7 @@ 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.Connector; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.MultiTableBatchWriter; @@ -132,6 +133,13 @@ public class ConnectorImpl extends Connector { } @Override + public ConditionalWriter createConditionalWriter(String tableName, Authorizations authorizations) throws TableNotFoundException { + ArgumentChecker.notNull(tableName, authorizations); + // TODO resolve table name to table id here and pass that + return new ConditionalWriterImpl(instance, credentials, getTableId(tableName), authorizations); + } + + @Override public Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException { ArgumentChecker.notNull(tableName, authorizations); return new ScannerImpl(instance, credentials, getTableId(tableName), authorizations); @@ -164,5 +172,4 @@ public class ConnectorImpl extends Connector { return instanceops; } - } http://git-wip-us.apache.org/repos/asf/accumulo/blob/49a7626c/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..40a9da4 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 @@ -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/49a7626c/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..4a405aa 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,7 @@ 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.Connector; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.MultiTableBatchWriter; @@ -128,4 +129,10 @@ public class MockConnector extends Connector { return new MockInstanceOperations(acu); } + @Override + public ConditionalWriter createConditionalWriter(String tableName, Authorizations authorizations) throws TableNotFoundException { + // TODO add implementation + throw new UnsupportedOperationException(); + } + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/49a7626c/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 names = new HashSet(); + HashSet prios = new HashSet(); + + 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/49a7626c/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..5b38559 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/data/ConditionalMutation.java @@ -0,0 +1,77 @@ +/* + * 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.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 conditions = new ArrayList(); + + 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); + } + + 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 getConditions() { + return conditions; + } + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/49a7626c/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, 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, SchemeFactory> schemes = new HashMap, 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 byName = new HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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/49a7626c/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; + } + } +}