You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2014/09/05 23:17:26 UTC

[10/18] ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java b/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
new file mode 100644
index 0000000..e0b17ac
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/TestProxySecurityOperations.java
@@ -0,0 +1,157 @@
+/*
+ * 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.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.proxy.thrift.SystemPermission;
+import org.apache.accumulo.proxy.thrift.TablePermission;
+import org.apache.accumulo.proxy.thrift.TimeType;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.server.TServer;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestProxySecurityOperations {
+  protected static TServer proxy;
+  protected static Thread thread;
+  protected static TestProxyClient tpc;
+  protected static ByteBuffer userpass;
+  protected static final int port = 10196;
+  protected static final String testtable = "testtable";
+  protected static final String testuser = "VonJines";
+  protected static final ByteBuffer testpw = ByteBuffer.wrap("fiveones".getBytes());
+  
+  @BeforeClass
+  public static void setup() throws Exception {
+    Properties prop = new Properties();
+    prop.setProperty("useMockInstance", "true");
+    prop.put("tokenClass", PasswordToken.class.getName());
+    
+    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
+        port, TCompactProtocol.Factory.class, prop);
+    thread = new Thread() {
+      @Override
+      public void run() {
+        proxy.serve();
+      }
+    };
+    thread.start();
+    
+    tpc = new TestProxyClient("localhost", port);
+    userpass = tpc.proxy().login("root", new TreeMap<String,String>() {
+      private static final long serialVersionUID = 1L;
+      
+      {
+        put("password", "");
+      }
+    });
+  }
+  
+  @AfterClass
+  public static void tearDown() throws InterruptedException {
+    proxy.stop();
+    thread.join();
+  }
+  
+  @Before
+  public void makeTestTableAndUser() throws Exception {
+    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
+    tpc.proxy().createLocalUser(userpass, testuser, testpw);
+  }
+  
+  @After
+  public void deleteTestTable() throws Exception {
+    tpc.proxy().deleteTable(userpass, testtable);
+    tpc.proxy().dropLocalUser(userpass, testuser);
+  }
+  
+  @Test
+  public void create() throws TException {
+    tpc.proxy().createLocalUser(userpass, testuser + "2", testpw);
+    assertTrue(tpc.proxy().listLocalUsers(userpass).contains(testuser + "2"));
+    tpc.proxy().dropLocalUser(userpass, testuser + "2");
+    assertTrue(!tpc.proxy().listLocalUsers(userpass).contains(testuser + "2"));
+  }
+  
+  @Test
+  public void authenticate() throws TException {
+    assertTrue(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(testpw)));
+    assertFalse(tpc.proxy().authenticateUser(userpass, "EvilUser", bb2pp(testpw)));
+    
+    tpc.proxy().changeLocalUserPassword(userpass, testuser, ByteBuffer.wrap("newpass".getBytes()));
+    assertFalse(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(testpw)));
+    assertTrue(tpc.proxy().authenticateUser(userpass, testuser, bb2pp(ByteBuffer.wrap("newpass".getBytes()))));
+    
+  }
+  
+  @Test
+  public void tablePermissions() throws TException {
+    tpc.proxy().grantTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE);
+    assertTrue(tpc.proxy().hasTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE));
+    
+    tpc.proxy().revokeTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE);
+    assertFalse(tpc.proxy().hasTablePermission(userpass, testuser, testtable, TablePermission.ALTER_TABLE));
+    
+  }
+  
+  @Test
+  public void systemPermissions() throws TException {
+    tpc.proxy().grantSystemPermission(userpass, testuser, SystemPermission.ALTER_USER);
+    assertTrue(tpc.proxy().hasSystemPermission(userpass, testuser, SystemPermission.ALTER_USER));
+    
+    tpc.proxy().revokeSystemPermission(userpass, testuser, SystemPermission.ALTER_USER);
+    assertFalse(tpc.proxy().hasSystemPermission(userpass, testuser, SystemPermission.ALTER_USER));
+    
+  }
+  
+  @Test
+  public void auths() throws TException {
+    HashSet<ByteBuffer> newauths = new HashSet<ByteBuffer>();
+    newauths.add(ByteBuffer.wrap("BBR".getBytes()));
+    newauths.add(ByteBuffer.wrap("Barney".getBytes()));
+    tpc.proxy().changeUserAuthorizations(userpass, testuser, newauths);
+    List<ByteBuffer> actualauths = tpc.proxy().getUserAuthorizations(userpass, testuser);
+    assertEquals(actualauths.size(), newauths.size());
+    
+    for (ByteBuffer auth : actualauths) {
+      assertTrue(newauths.contains(auth));
+    }
+  }
+  
+  private Map<String,String> bb2pp(ByteBuffer cf) {
+    Map<String,String> toRet = new TreeMap<String,String>();
+    toRet.put("password", ByteBufferUtil.toString(cf));
+    return toRet;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java b/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
new file mode 100644
index 0000000..87d3454
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/proxy/TestProxyTableOperations.java
@@ -0,0 +1,212 @@
+/*
+ * 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.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.proxy.thrift.ColumnUpdate;
+import org.apache.accumulo.proxy.thrift.TimeType;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.server.TServer;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestProxyTableOperations {
+  
+  protected static TServer proxy;
+  protected static Thread thread;
+  protected static TestProxyClient tpc;
+  protected static ByteBuffer userpass;
+  protected static final int port = 10195;
+  protected static final String testtable = "testtable";
+  
+  @SuppressWarnings("serial")
+  @BeforeClass
+  public static void setup() throws Exception {
+    Properties prop = new Properties();
+    prop.setProperty("useMockInstance", "true");
+    prop.put("tokenClass", PasswordToken.class.getName());
+    
+    proxy = Proxy.createProxyServer(Class.forName("org.apache.accumulo.proxy.thrift.AccumuloProxy"), Class.forName("org.apache.accumulo.proxy.ProxyServer"),
+        port, TCompactProtocol.Factory.class, prop);
+    thread = new Thread() {
+      @Override
+      public void run() {
+        proxy.serve();
+      }
+    };
+    thread.start();
+    tpc = new TestProxyClient("localhost", port);
+    userpass = tpc.proxy().login("root", new TreeMap<String,String>() {
+      {
+        put("password", "");
+      }
+    });
+  }
+  
+  @AfterClass
+  public static void tearDown() throws InterruptedException {
+    proxy.stop();
+    thread.join();
+  }
+  
+  @Before
+  public void makeTestTable() throws Exception {
+    tpc.proxy().createTable(userpass, testtable, true, TimeType.MILLIS);
+  }
+  
+  @After
+  public void deleteTestTable() throws Exception {
+    tpc.proxy().deleteTable(userpass, testtable);
+  }
+  
+  @Test
+  public void createExistsDelete() throws TException {
+    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().createTable(userpass, "testtable2", true, TimeType.MILLIS);
+    assertTrue(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().deleteTable(userpass, "testtable2");
+    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
+  }
+  
+  @Test
+  public void listRename() throws TException {
+    assertFalse(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().renameTable(userpass, testtable, "testtable2");
+    assertTrue(tpc.proxy().tableExists(userpass, "testtable2"));
+    tpc.proxy().renameTable(userpass, "testtable2", testtable);
+    assertTrue(tpc.proxy().listTables(userpass).contains("testtable"));
+    
+  }
+  
+  // This test does not yet function because the backing Mock instance does not yet support merging
+  @Test
+  public void merge() throws TException {
+    Set<ByteBuffer> splits = new HashSet<ByteBuffer>();
+    splits.add(ByteBuffer.wrap("a".getBytes()));
+    splits.add(ByteBuffer.wrap("c".getBytes()));
+    splits.add(ByteBuffer.wrap("z".getBytes()));
+    tpc.proxy().addSplits(userpass, testtable, splits);
+    
+    tpc.proxy().mergeTablets(userpass, testtable, ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap("d".getBytes()));
+    
+    splits.remove(ByteBuffer.wrap("c".getBytes()));
+    
+    List<ByteBuffer> tableSplits = tpc.proxy().listSplits(userpass, testtable, 10);
+    
+    for (ByteBuffer split : tableSplits)
+      assertTrue(splits.contains(split));
+    assertTrue(tableSplits.size() == splits.size());
+    
+  }
+  
+  @Test
+  public void splits() throws TException {
+    Set<ByteBuffer> splits = new HashSet<ByteBuffer>();
+    splits.add(ByteBuffer.wrap("a".getBytes()));
+    splits.add(ByteBuffer.wrap("b".getBytes()));
+    splits.add(ByteBuffer.wrap("z".getBytes()));
+    tpc.proxy().addSplits(userpass, testtable, splits);
+    
+    List<ByteBuffer> tableSplits = tpc.proxy().listSplits(userpass, testtable, 10);
+    
+    for (ByteBuffer split : tableSplits)
+      assertTrue(splits.contains(split));
+    assertTrue(tableSplits.size() == splits.size());
+  }
+  
+  @Test
+  public void constraints() throws TException {
+    int cid = tpc.proxy().addConstraint(userpass, testtable, "org.apache.accumulo.TestConstraint");
+    Map<String,Integer> constraints = tpc.proxy().listConstraints(userpass, testtable);
+    assertEquals((int) constraints.get("org.apache.accumulo.TestConstraint"), cid);
+    tpc.proxy().removeConstraint(userpass, testtable, cid);
+    constraints = tpc.proxy().listConstraints(userpass, testtable);
+    assertNull(constraints.get("org.apache.accumulo.TestConstraint"));
+  }
+  
+  @Test
+  public void localityGroups() throws TException {
+    Map<String,Set<String>> groups = new HashMap<String,Set<String>>();
+    Set<String> group1 = new HashSet<String>();
+    group1.add("cf1");
+    groups.put("group1", group1);
+    Set<String> group2 = new HashSet<String>();
+    group2.add("cf2");
+    group2.add("cf3");
+    groups.put("group2", group2);
+    tpc.proxy().setLocalityGroups(userpass, testtable, groups);
+    
+    Map<String,Set<String>> actualGroups = tpc.proxy().getLocalityGroups(userpass, testtable);
+    
+    assertEquals(groups.size(), actualGroups.size());
+    for (String groupName : groups.keySet()) {
+      assertTrue(actualGroups.containsKey(groupName));
+      assertEquals(groups.get(groupName).size(), actualGroups.get(groupName).size());
+      for (String cf : groups.get(groupName)) {
+        assertTrue(actualGroups.get(groupName).contains(cf));
+      }
+    }
+  }
+  
+  @Test
+  public void tableProperties() throws TException {
+    tpc.proxy().setTableProperty(userpass, testtable, "test.property1", "wharrrgarbl");
+    assertEquals(tpc.proxy().getTableProperties(userpass, testtable).get("test.property1"), "wharrrgarbl");
+    tpc.proxy().removeTableProperty(userpass, testtable, "test.property1");
+    assertNull(tpc.proxy().getTableProperties(userpass, testtable).get("test.property1"));
+  }
+  
+  private static void addMutation(Map<ByteBuffer,List<ColumnUpdate>> mutations, String row, String cf, String cq, String value) {
+    ColumnUpdate update = new ColumnUpdate(ByteBuffer.wrap(cf.getBytes()), ByteBuffer.wrap(cq.getBytes()));
+    update.setValue(value.getBytes());
+    mutations.put(ByteBuffer.wrap(row.getBytes()), Collections.singletonList(update));
+  }
+  
+  @Test
+  public void tableOperationsRowMethods() throws TException {
+    Map<ByteBuffer,List<ColumnUpdate>> mutations = new HashMap<ByteBuffer,List<ColumnUpdate>>();
+    for (int i = 0; i < 10; i++) {
+      addMutation(mutations, "" + i, "cf", "cq", "");
+    }
+    tpc.proxy().updateAndFlush(userpass, testtable, mutations);
+    
+    assertEquals(tpc.proxy().getMaxRow(userpass, testtable, null, null, true, null, true), ByteBuffer.wrap("9".getBytes()));
+    
+    tpc.proxy().deleteRows(userpass, testtable, ByteBuffer.wrap("51".getBytes()), ByteBuffer.wrap("99".getBytes()));
+    assertEquals(tpc.proxy().getMaxRow(userpass, testtable, null, null, true, null, true), ByteBuffer.wrap("5".getBytes()));
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
index 8f6b830..6ee671e 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
@@ -57,7 +57,7 @@ public class BloomFilterIT extends ConfigurableMacIT {
     siteConfig.put(Property.TABLE_BLOOM_SIZE.getKey(), "2000000");
     siteConfig.put(Property.TABLE_BLOOM_ERRORRATE.getKey(), "1%");
     siteConfig.put(Property.TABLE_BLOOM_LOAD_THRESHOLD.getKey(), "0");
-    siteConfig.put(Property.TSERV_MUTATION_QUEUE_MAX.getKey(), "10M");
+    siteConfig.put(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(), "10M");
     siteConfig.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), "64K");
     cfg.setSiteConfig(siteConfig );
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 526f962..4632083 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -165,7 +165,8 @@ public class DurabilityIT extends ConfigurableMacIT {
     c.tableOperations().create(tableName);
     props = map(c.tableOperations().getProperties(tableName));
     assertEquals("none", props.get(Property.TABLE_DURABILITY.getKey()));
-
+    restartTServer();
+    assertTrue(c.tableOperations().exists(tableName));
   }
 
   private long readSome(String table) throws Exception {