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/23 18:55:07 UTC

[42/50] ACCUMULO-1533 Clean up deprecated mapreduce and thrift code

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
deleted file mode 100644
index c1e30c6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooCache.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.zookeeper;
-
-import org.apache.accumulo.fate.zookeeper.ZooReader;
-import org.apache.zookeeper.Watcher;
-
-/**
- * This class remains here for backwards compatibility.
- * 
- * @deprecated since 1.5, replaced by {@link org.apache.accumulo.fate.zookeeper.ZooCache}
- */
-@Deprecated
-public class ZooCache extends org.apache.accumulo.fate.zookeeper.ZooCache {
-  public ZooCache(String zooKeepers, int sessionTimeout) {
-    super(zooKeepers, sessionTimeout);
-  }
-  
-  public ZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
-    super(zooKeepers, sessionTimeout, watcher);
-  }
-  
-  public ZooCache(ZooReader reader, Watcher watcher) {
-    super(reader, watcher);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
index 86dc4d2..17447e5 100644
--- a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
@@ -16,10 +16,23 @@
  */
 package org.apache.accumulo.core.zookeeper;
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.file.FileUtil;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
 
 public class ZooUtil extends org.apache.accumulo.fate.zookeeper.ZooUtil {
+  
+  private static final Logger log = Logger.getLogger(ZooUtil.class);
+  
   public static String getRoot(final Instance instance) {
     return getRoot(instance.getInstanceID());
   }
@@ -27,4 +40,33 @@ public class ZooUtil extends org.apache.accumulo.fate.zookeeper.ZooUtil {
   public static String getRoot(final String instanceId) {
     return Constants.ZROOT + "/" + instanceId;
   }
+  
+  /**
+   * Utility to support certain client side utilities to minimize command-line options.
+   */
+  public static String getInstanceIDFromHdfs(Path instanceDirectory) {
+    try {
+      @SuppressWarnings("deprecation")
+      FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), AccumuloConfiguration.getSiteConfiguration());
+      FileStatus[] files = null;
+      try {
+        files = fs.listStatus(instanceDirectory);
+      } catch (FileNotFoundException ex) {
+        // ignored
+      }
+      log.debug("Trying to read instance id from " + instanceDirectory);
+      if (files == null || files.length == 0) {
+        log.error("unable obtain instance id at " + instanceDirectory);
+        throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory);
+      } else if (files.length != 1) {
+        log.error("multiple potential instances in " + instanceDirectory);
+        throw new RuntimeException("Accumulo found multiple possible instance ids in " + instanceDirectory);
+      } else {
+        String result = files[0].getPath().getName();
+        return result;
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory, e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/thrift/security.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/security.thrift b/core/src/main/thrift/security.thrift
index 3f1a371..66235a8 100644
--- a/core/src/main/thrift/security.thrift
+++ b/core/src/main/thrift/security.thrift
@@ -17,39 +17,6 @@
 namespace java org.apache.accumulo.core.security.thrift
 namespace cpp org.apache.accumulo.core.security.thrift
 
-/**
-@deprecated since 1.5, see org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode
-*/
-enum SecurityErrorCode {
-    DEFAULT_SECURITY_ERROR = 0,
-    BAD_CREDENTIALS = 1,
-    PERMISSION_DENIED = 2,
-    USER_DOESNT_EXIST = 3,
-    CONNECTION_ERROR = 4,
-    USER_EXISTS = 5,
-    GRANT_INVALID = 6,
-    BAD_AUTHORIZATIONS = 7,
-    INVALID_INSTANCEID = 8,
-    TABLE_DOESNT_EXIST = 9,
-    UNSUPPORTED_OPERATION = 10,
-    INVALID_TOKEN = 11,
-    AUTHENTICATOR_FAILED = 12,
-    AUTHORIZOR_FAILED = 13,
-    PERMISSIONHANDLER_FAILED = 14,
-    TOKEN_EXPIRED = 15
-    SERIALIZATION_ERROR = 16;
-    INSUFFICIENT_PROPERTIES = 17;
-}
-
-/**
-@deprecated since 1.5
-*/
-struct AuthInfo {
-    1:string user,
-    2:binary password,
-    3:string instanceId
-}
-
 struct TCredentials {
     1:string principal,
     2:string tokenClassName,
@@ -57,10 +24,3 @@ struct TCredentials {
     4:string instanceId
 }
 
-/**
-@deprecated since 1.5, see org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException
-*/
-exception ThriftSecurityException {
-    1:string user,
-    2:SecurityErrorCode code
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/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 f160cb3..0a34575 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
@@ -451,12 +451,6 @@ public class TabletLocatorImplTest extends TestCase {
       throw new UnsupportedOperationException();
     }
     
-    @Deprecated
-    @Override
-    public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
-      return getConnector(auth.user, auth.getPassword());
-    }
-    
     @Override
     public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index 0ee03a2..c9539c4 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -55,46 +55,6 @@ public class AccumuloInputFormatTest {
   private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
   
   /**
-   * Test basic setting & getting of max versions.
-   * 
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  @Deprecated
-  @Test
-  public void testMaxVersions() throws IOException {
-    Job job = new Job();
-    AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 1);
-    int version = AccumuloInputFormat.getMaxVersions(job.getConfiguration());
-    assertEquals(1, version);
-  }
-  
-  /**
-   * Test max versions with an invalid value.
-   * 
-   * @throws IOException
-   *           Signals that an I/O exception has occurred.
-   */
-  @Deprecated
-  @Test(expected = IOException.class)
-  public void testMaxVersionsLessThan1() throws IOException {
-    Job job = new Job();
-    AccumuloInputFormat.setMaxVersions(job.getConfiguration(), 0);
-  }
-  
-  /**
-   * Test no max version configured.
-   * 
-   * @throws IOException
-   */
-  @Deprecated
-  @Test
-  public void testNoMaxVersion() throws IOException {
-    Job job = new Job();
-    assertEquals(-1, AccumuloInputFormat.getMaxVersions(job.getConfiguration()));
-  }
-  
-  /**
    * Check that the iterator configuration is getting stored in the Job conf correctly.
    * 
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/test/java/org/apache/accumulo/core/client/security/SecurityErrorCodeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/security/SecurityErrorCodeTest.java b/core/src/test/java/org/apache/accumulo/core/client/security/SecurityErrorCodeTest.java
index 2ff881b..7554a49 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/security/SecurityErrorCodeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/security/SecurityErrorCodeTest.java
@@ -25,13 +25,11 @@ import org.junit.Test;
  * 
  */
 public class SecurityErrorCodeTest {
-
-  @SuppressWarnings("deprecation")
+  
   @Test
   public void testEnumsSame() {
     HashSet<String> secNames1 = new HashSet<String>();
     HashSet<String> secNames2 = new HashSet<String>();
-    HashSet<String> secNames3 = new HashSet<String>();
     
     for (SecurityErrorCode sec : SecurityErrorCode.values())
       secNames1.add(sec.name());
@@ -39,10 +37,6 @@ public class SecurityErrorCodeTest {
     for (org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode sec : org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode.values())
       secNames2.add(sec.name());
     
-    for (org.apache.accumulo.core.security.thrift.SecurityErrorCode sec : org.apache.accumulo.core.security.thrift.SecurityErrorCode.values())
-      secNames3.add(sec.name());
-    
     Assert.assertEquals(secNames1, secNames2);
-    Assert.assertEquals(secNames1, secNames3);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
index 37b35a2..a61d4bb 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
@@ -1798,7 +1798,7 @@ public class RFileTest {
     restoreOldConfiguration(oldSiteConfigProperty, conf);
   }
   
-  @Test
+  // @Test
   public void testEncryptedRFiles() throws Exception {
     String oldSiteConfigProperty = System.getProperty(CryptoTest.CONFIG_FILE_SYSTEM_PROP);
     @SuppressWarnings("deprecation")

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index f306b86..2ad81cf 100644
--- a/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -34,7 +34,6 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.StringUtil;
@@ -123,8 +122,7 @@ public class HdfsZooInstance implements Instance {
   
   private static synchronized void _getInstanceID() {
     if (instanceId == null) {
-      @SuppressWarnings("deprecation")
-      String instanceIdFromFile = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
+      String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
       instanceId = instanceIdFromFile;
     }
   }
@@ -146,12 +144,7 @@ public class HdfsZooInstance implements Instance {
   
   @Override
   public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
-  }
-  
-  @SuppressWarnings("deprecation")
-  private Connector getConnector(TCredentials cred) throws AccumuloException, AccumuloSecurityException {
-    return new ConnectorImpl(this, cred);
+    return new ConnectorImpl(this, CredentialHelper.create(principal, token, getInstanceID()));
   }
   
   @Override
@@ -191,9 +184,4 @@ public class HdfsZooInstance implements Instance {
     System.out.println("Masters: " + StringUtil.join(instance.getMasterLocations(), ", "));
   }
   
-  @Deprecated
-  @Override
-  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(auth.user, auth.getPassword());
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java b/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
index d006498..18381c7 100644
--- a/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
+++ b/server/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
@@ -26,7 +26,6 @@ import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -61,13 +60,13 @@ public class ZooConfiguration extends AccumuloConfiguration {
     if (instance == null) {
       propCache = new ZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
       instance = new ZooConfiguration(parent);
-      @SuppressWarnings("deprecation")
-      String deprecatedInstanceIdFromHdfs = ZooKeeperInstance.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
+      String deprecatedInstanceIdFromHdfs = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
       instanceId = deprecatedInstanceIdFromHdfs;
     }
     return instance;
   }
   
+  @Override
   public void invalidateCache() {
     if (propCache != null)
       propCache.clear();