You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2014/04/21 22:10:14 UTC

[2/4] git commit: ACCUMULO-2700 Fix system credentials checks and add a test

ACCUMULO-2700 Fix system credentials checks and add a test


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/d2750908
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/d2750908
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/d2750908

Branch: refs/heads/master
Commit: d27509084ff45cef892c5735ee8fb559cd61dc0c
Parents: e4b08b1
Author: Christopher Tubbs <ct...@apache.org>
Authored: Mon Apr 21 15:14:05 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Mon Apr 21 15:27:53 2014 -0400

----------------------------------------------------------------------
 .../server/security/SecurityOperation.java      |   8 +-
 .../server/security/SystemCredentials.java      |  51 ++++---
 .../server/security/SystemCredentialsIT.java    | 148 +++++++++++++++++++
 3 files changed, 177 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2750908/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index f8b1702..c2a7001 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -154,11 +154,11 @@ public class SecurityOperation {
     if (!credentials.getInstanceId().equals(HdfsZooInstance.getInstance().getInstanceID()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.INVALID_INSTANCEID);
 
+    AuthenticationToken token = AuthenticationTokenSerializer.deserialize(credentials.getTokenClassName(), credentials.getToken());
     if (isSystemUser(credentials)) {
-      authenticateSystemUser(credentials);
+      authenticateSystemUserToken(credentials, token);
     } else {
       try {
-        AuthenticationToken token = AuthenticationTokenSerializer.deserialize(credentials.getTokenClassName(), credentials.getToken());
         if (!authenticator.authenticateUser(credentials.getPrincipal(), token)) {
           throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
         }
@@ -169,8 +169,8 @@ public class SecurityOperation {
     }
   }
 
-  private void authenticateSystemUser(TCredentials credentials) throws ThriftSecurityException {
-    if (SystemCredentials.get().getToken().equals(credentials.getToken()))
+  private void authenticateSystemUserToken(TCredentials credentials, AuthenticationToken token) throws ThriftSecurityException {
+    if (!SystemCredentials.get().getToken().equals(token))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2750908/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java b/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
index 9b4931d..b5d7aba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
@@ -43,55 +43,55 @@ import org.apache.hadoop.io.Writable;
  * @since 1.6.0
  */
 public final class SystemCredentials extends Credentials {
-  
+
   private static final SecurityPermission SYSTEM_CREDENTIALS_PERMISSION = new SecurityPermission("systemCredentialsPermission");
-  
+
   private static SystemCredentials SYSTEM_CREDS = null;
   private static final String SYSTEM_PRINCIPAL = "!SYSTEM";
-  private static final SystemToken SYSTEM_TOKEN = SystemToken.get();
-  
+
   private final TCredentials AS_THRIFT;
-  
-  private SystemCredentials() {
-    super(SYSTEM_PRINCIPAL, SYSTEM_TOKEN);
-    AS_THRIFT = super.toThrift(HdfsZooInstance.getInstance());
+
+  SystemCredentials(Instance instance) {
+    super(SYSTEM_PRINCIPAL, SystemToken.get(instance));
+    AS_THRIFT = super.toThrift(instance);
   }
-  
+
   public static SystemCredentials get() {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
       sm.checkPermission(SYSTEM_CREDENTIALS_PERMISSION);
     }
     if (SYSTEM_CREDS == null) {
-      SYSTEM_CREDS = new SystemCredentials();
+      SYSTEM_CREDS = new SystemCredentials(HdfsZooInstance.getInstance());
     }
     return SYSTEM_CREDS;
   }
-  
+
   @Override
   public TCredentials toThrift(Instance instance) {
     if (!AS_THRIFT.getInstanceId().equals(instance.getInstanceID()))
       throw new IllegalArgumentException("Unexpected instance used for " + SystemCredentials.class.getSimpleName() + ": " + instance.getInstanceID());
     return AS_THRIFT;
   }
-  
+
   /**
    * An {@link AuthenticationToken} type for Accumulo servers for inter-server communication.
    * 
    * @since 1.6.0
    */
   public static final class SystemToken extends PasswordToken {
-    
+
     /**
      * A Constructor for {@link Writable}.
      */
     public SystemToken() {}
-    
+
     private SystemToken(byte[] systemPassword) {
       super(systemPassword);
     }
-    
-    private static SystemToken get() {
+
+    private static SystemToken get(Instance instance) {
+      byte[] instanceIdBytes = instance.getInstanceID().getBytes(Constants.UTF8);
       byte[] confChecksum;
       MessageDigest md;
       try {
@@ -99,11 +99,11 @@ public final class SystemCredentials extends Credentials {
       } catch (NoSuchAlgorithmException e) {
         throw new RuntimeException("Failed to compute configuration checksum", e);
       }
-      
+
       // seed the config with the version and instance id, so at least it's not empty
       md.update(ServerConstants.WIRE_VERSION.toString().getBytes(Constants.UTF8));
-      md.update(HdfsZooInstance.getInstance().getInstanceID().getBytes(Constants.UTF8));
-      
+      md.update(instanceIdBytes);
+
       for (Entry<String,String> entry : ServerConfiguration.getSiteConfiguration()) {
         // only include instance properties
         if (entry.getKey().startsWith(Property.INSTANCE_PREFIX.toString())) {
@@ -112,16 +112,15 @@ public final class SystemCredentials extends Credentials {
         }
       }
       confChecksum = md.digest();
-      
+
       int wireVersion = ServerConstants.WIRE_VERSION;
-      byte[] inst = HdfsZooInstance.getInstance().getInstanceID().getBytes(Constants.UTF8);
-      
-      ByteArrayOutputStream bytes = new ByteArrayOutputStream(3 * (Integer.SIZE / Byte.SIZE) + inst.length + confChecksum.length);
+
+      ByteArrayOutputStream bytes = new ByteArrayOutputStream(3 * (Integer.SIZE / Byte.SIZE) + instanceIdBytes.length + confChecksum.length);
       DataOutputStream out = new DataOutputStream(bytes);
       try {
         out.write(wireVersion * -1);
-        out.write(inst.length);
-        out.write(inst);
+        out.write(instanceIdBytes.length);
+        out.write(instanceIdBytes);
         out.write(confChecksum.length);
         out.write(confChecksum);
       } catch (IOException e) {
@@ -131,5 +130,5 @@ public final class SystemCredentials extends Credentials {
       return new SystemToken(Base64.encodeBase64(bytes.toByteArray()));
     }
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2750908/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java b/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java
new file mode 100644
index 0000000..f5296c0
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.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.server.security;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.security.SecurityErrorCode;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.junit.Test;
+
+public class SystemCredentialsIT extends ConfigurableMacIT {
+
+  private static final int FAIL_CODE = 7;
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 1 * 60;
+  }
+
+  @Test
+  public void testSystemCredentials() throws Exception {
+    assertEquals(0, exec(SystemCredentialsIT.class, "good", getCluster().getZooKeepers()).waitFor());
+    assertEquals(FAIL_CODE, exec(SystemCredentialsIT.class, "bad", getCluster().getZooKeepers()).waitFor());
+  }
+
+  public static void main(final String[] args) throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
+    Credentials creds = null;
+    if (args.length < 2)
+      throw new RuntimeException("Incorrect usage; expected to be run by test only");
+    if (args[0].equals("bad")) {
+      creds = new SystemCredentials(new Instance() {
+
+        @Deprecated
+        @Override
+        public void setConfiguration(AccumuloConfiguration conf) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int getZooKeepersSessionTimeOut() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public String getZooKeepers() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public String getRootTabletLocation() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public List<String> getMasterLocations() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public String getInstanceName() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public String getInstanceID() {
+          return SystemCredentials.class.getName();
+        }
+
+        @Override
+        public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+          throw new UnsupportedOperationException();
+        }
+
+        @Deprecated
+        @Override
+        public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
+          throw new UnsupportedOperationException();
+        }
+
+        @Deprecated
+        @Override
+        public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
+          throw new UnsupportedOperationException();
+        }
+
+        @Deprecated
+        @Override
+        public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
+          throw new UnsupportedOperationException();
+        }
+
+        @Deprecated
+        @Override
+        public AccumuloConfiguration getConfiguration() {
+          throw new UnsupportedOperationException();
+        }
+      });
+    } else if (args[0].equals("good")) {
+      creds = SystemCredentials.get();
+    }
+    Instance instance = HdfsZooInstance.getInstance();
+    Connector conn = instance.getConnector(creds.getPrincipal(), creds.getToken());
+    try {
+      Scanner scan = conn.createScanner(RootTable.NAME, Authorizations.EMPTY);
+      for (Entry<Key,Value> e : scan) {
+        e.hashCode();
+      }
+    } catch (RuntimeException e) {
+      // catch the runtime exception from the scanner iterator
+      if (e.getCause() instanceof AccumuloSecurityException && ((AccumuloSecurityException) e.getCause()).getSecurityErrorCode() == SecurityErrorCode.BAD_CREDENTIALS) {
+          e.printStackTrace(System.err);
+          System.exit(FAIL_CODE);
+      }
+    }
+  }
+}