You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bi...@apache.org on 2011/10/18 23:19:50 UTC

svn commit: r1185873 - in /incubator/accumulo/trunk/src/core/src: main/java/org/apache/accumulo/core/client/mock/ test/java/org/apache/accumulo/core/client/mock/

Author: billie
Date: Tue Oct 18 21:19:50 2011
New Revision: 1185873

URL: http://svn.apache.org/viewvc?rev=1185873&view=rev
Log: (empty)

Added:
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java
Modified:
    incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
    incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java

Added: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java?rev=1185873&view=auto
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java (added)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java Tue Oct 18 21:19:50 2011
@@ -0,0 +1,72 @@
+package org.apache.accumulo.core.client.mock;
+
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+
+/**
+ * {@link BatchDeleter} for a {@link MockAccumulo} instance. Behaves similarly
+ * to a regular {@link BatchDeleter}, with a few exceptions:
+ * <ol>
+ * <li>There is no waiting for memory to fill before flushing</li>
+ * <li>Only one thread is used for writing</li>
+ * </ol>
+ * 
+ * Otherwise, it behaves as expected.
+ */
+public class MockBatchDeleter extends MockBatchScanner implements BatchDeleter {
+
+    private final MockAccumulo acc;
+    private final String tableName;
+    private final Authorizations authorizations;
+
+    /**
+     * Create a {@link BatchDeleter} for the specified instance on the specified
+     * table where the writer uses the specified {@link Authorizations}.
+     * @param acc
+     * @param tableName
+     * @param auths
+     */
+    public MockBatchDeleter(MockAccumulo acc, String tableName,
+            Authorizations auths) {
+        super(acc.tables.get(tableName), auths);
+        this.authorizations = auths;
+        this.acc = acc;
+        this.tableName = tableName;
+    }
+
+    @Override
+    public void delete() throws MutationsRejectedException,
+            TableNotFoundException {
+
+        BatchWriter writer = new MockBatchWriter(acc, tableName);
+        try {
+            // super.setScanIterators(Integer.MAX_VALUE,
+            // SortedKeyIterator.class.getName(), BatchDeleter.class.getName()
+            // + ".NOVALUE");
+
+            Iterator<Entry<Key, Value>> iter = super.iterator();
+            while (iter.hasNext()) {
+                Entry<Key, Value> next = iter.next();
+                Key k = next.getKey();
+                Mutation m = new Mutation(k.getRow());
+                m.putDelete(k.getColumnFamily(), k.getColumnQualifier(),
+                        new ColumnVisibility(k.getColumnVisibility()),
+                        k.getTimestamp());
+                writer.addMutation(m);
+            }
+        } finally {
+            writer.close();
+        }
+    }
+
+}

Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java?rev=1185873&r1=1185872&r2=1185873&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java (original)
+++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java Tue Oct 18 21:19:50 2011
@@ -53,7 +53,7 @@ public class MockConnector extends Conne
     @Override
     public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations, int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
             throws TableNotFoundException {
-        return null;
+    return new MockBatchDeleter(acu, tableName, authorizations);
     }
 
     @Override

Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java?rev=1185873&r1=1185872&r2=1185873&view=diff
==============================================================================
--- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java (original)
+++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java Tue Oct 18 21:19:50 2011
@@ -26,13 +26,12 @@ import java.util.Map.Entry;
 import java.util.Random;
 
 import org.apache.accumulo.core.Constants;
+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.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mock.MockConnector;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -154,6 +153,72 @@ public class MockConnectorTest  {
     }
     
     @Test
+    public void testDeletewithBatchDeleter() throws Exception {
+        Connector c = new MockConnector("root");
+
+        // make sure we are using a clean table
+        if (c.tableOperations().exists("test"))
+            c.tableOperations().delete("test");
+        c.tableOperations().create("test");
+
+        BatchDeleter deleter = c.createBatchDeleter("test", Constants.NO_AUTHS,
+                2, 10000L, 1000L, 4);
+        // first make sure it deletes fine when its empty
+        deleter.setRanges(Collections.singletonList(new Range(("r1"))));
+        deleter.delete();
+        this.checkRemaining(c, "test", 0);
+
+        // test deleting just one row
+        BatchWriter writer = c.createBatchWriter("test", 10, 10, 1);
+        Mutation m = new Mutation("r1");
+        m.put("fam", "qual", "value");
+        writer.addMutation(m);
+
+        // make sure the write goes through
+        writer.flush();
+        writer.close();
+
+        deleter.setRanges(Collections.singletonList(new Range(("r1"))));
+        deleter.delete();
+        this.checkRemaining(c, "test", 0);
+
+        // test multi row deletes
+        writer = c.createBatchWriter("test", 10, 10, 1);
+        m = new Mutation("r1");
+        m.put("fam", "qual", "value");
+        writer.addMutation(m);
+        Mutation m2 = new Mutation("r2");
+        m2.put("fam", "qual", "value");
+        writer.addMutation(m2);
+
+        // make sure the write goes through
+        writer.flush();
+        writer.close();
+
+        deleter.setRanges(Collections.singletonList(new Range(("r1"))));
+        deleter.delete();
+        checkRemaining(c, "test", 1);
+    }
+
+    /**
+     * Test to make sure that a certain number of rows remain
+     * @param c connector to the {@link MockInstance}
+     * @param tableName TODO
+     * @param count number of entries to expect in the table
+     * @param count TODO
+     */
+    private void checkRemaining(Connector c, String tableName, int count) throws Exception {
+        Scanner scanner = c.createScanner(tableName, Constants.NO_AUTHS);
+
+        int total = 0;
+        for (@SuppressWarnings("unused")
+        Entry<Key, Value> entry : scanner) {
+            total++;
+        }
+        assertEquals(count, total);
+    }
+
+    @Test
     public void testCMod() throws Exception {
     	//test writing to a table that the is being scanned
     	Connector c = new MockConnector("root");