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 2D5C8DB97 for ; Fri, 25 Jan 2013 07:05:39 +0000 (UTC) Received: (qmail 87085 invoked by uid 500); 25 Jan 2013 07:05:37 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 87032 invoked by uid 500); 25 Jan 2013 07:05:36 -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 86992 invoked by uid 99); 25 Jan 2013 07:05:35 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Jan 2013 07:05:35 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Jan 2013 07:05:30 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 880D42388C27; Fri, 25 Jan 2013 07:04:38 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1438354 [10/12] - in /accumulo/trunk: ./ assemble/ bin/ conf/examples/1GB/native-standalone/ conf/examples/1GB/standalone/ conf/examples/2GB/native-standalone/ conf/examples/2GB/standalone/ conf/examples/3GB/native-standalone/ conf/example... Date: Fri, 25 Jan 2013 07:04:29 -0000 To: commits@accumulo.apache.org From: vines@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130125070438.880D42388C27@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/monitor/servlets/TServersServlet.java Fri Jan 25 07:04:25 2013 @@ -126,9 +126,9 @@ public class TServersServlet extends Bas TabletClientService.Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, Monitor.getSystemConfiguration()); try { for (String tableId : Monitor.getMmi().tableMap.keySet()) { - tsStats.addAll(client.getTabletStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials(), tableId)); + tsStats.addAll(client.getTabletStats(Tracer.traceInfo(), SecurityConstants.getThriftSystemCredentials(), tableId)); } - historical = client.getHistoricalStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials()); + historical = client.getHistoricalStats(Tracer.traceInfo(), SecurityConstants.getThriftSystemCredentials()); } finally { ThriftUtil.returnClient(client); } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java Fri Jan 25 07:04:25 2013 @@ -21,7 +21,6 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.security.SecurityPermission; @@ -30,7 +29,10 @@ import java.util.Map.Entry; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper; +import org.apache.accumulo.core.security.tokens.AccumuloToken; +import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper; +import org.apache.accumulo.core.security.tokens.UserPassToken; import org.apache.accumulo.server.client.HdfsZooInstance; import org.apache.accumulo.server.conf.ServerConfiguration; import org.apache.accumulo.server.master.state.TabletServerState; @@ -41,11 +43,11 @@ public class SecurityConstants { public static final String SYSTEM_USERNAME = "!SYSTEM"; private static final byte[] SYSTEM_PASSWORD = makeSystemPassword(); - private static final AuthInfo systemCredentials = new AuthInfo(SYSTEM_USERNAME, ByteBuffer.wrap(SYSTEM_PASSWORD), HdfsZooInstance.getInstance() - .getInstanceID()); + private static final AccumuloToken systemToken = new UserPassToken(SYSTEM_USERNAME, SYSTEM_PASSWORD); + private static final InstanceTokenWrapper systemCredentials = new InstanceTokenWrapper(systemToken, HdfsZooInstance.getInstance().getInstanceID()); public static byte[] confChecksum = null; - public static AuthInfo getSystemCredentials() { + public static InstanceTokenWrapper getSystemCredentials() { SecurityManager sm = System.getSecurityManager(); if (sm != null) { sm.checkPermission(SYSTEM_CREDENTIALS_PERMISSION); @@ -53,6 +55,10 @@ public class SecurityConstants { return systemCredentials; } + public static ThriftInstanceTokenWrapper getThriftSystemCredentials() { + return systemCredentials.toThrift(); + } + private static byte[] makeSystemPassword() { byte[] version = Constants.VERSION.getBytes(); byte[] inst = HdfsZooInstance.getInstance().getInstanceID().getBytes(); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Fri Jan 25 07:04:25 2013 @@ -86,7 +86,7 @@ import org.apache.accumulo.core.iterator import org.apache.accumulo.core.master.thrift.TabletLoadState; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.ColumnVisibility; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper; import org.apache.accumulo.core.util.CachedConfiguration; import org.apache.accumulo.core.util.LocalityGroupUtil; import org.apache.accumulo.core.util.MetadataTable.DataFileValue; @@ -681,10 +681,10 @@ public class Tablet { } synchronized (bulkFileImportLock) { - AuthInfo auths = SecurityConstants.getSystemCredentials(); + InstanceTokenWrapper auths = SecurityConstants.getSystemCredentials(); Connector conn; try { - conn = HdfsZooInstance.getInstance().getConnector(auths.user, auths.password); + conn = HdfsZooInstance.getInstance().getConnector(auths); } catch (Exception ex) { throw new IOException(ex); } @@ -846,7 +846,7 @@ public class Tablet { // need to write to !METADATA before writing to walog, when things are done in the reverse order // data could be lost... the minor compaction start even should be written before the following metadata // write is made - AuthInfo creds = SecurityConstants.getSystemCredentials(); + InstanceTokenWrapper creds = SecurityConstants.getSystemCredentials(); synchronized (timeLock) { if (commitSession.getMaxCommittedTime() > persistedTime) @@ -2270,7 +2270,7 @@ public class Tablet { } if (updateMetadata) { - AuthInfo creds = SecurityConstants.getSystemCredentials(); + InstanceTokenWrapper creds = SecurityConstants.getSystemCredentials(); // if multiple threads were allowed to update this outside of a sync block, then it would be // a race condition MetadataTable.updateTabletFlushID(extent, tableFlushID, creds, tabletServer.getLock()); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Fri Jan 25 07:04:25 2013 @@ -106,11 +106,11 @@ import org.apache.accumulo.core.master.t import org.apache.accumulo.core.master.thrift.TabletLoadState; import org.apache.accumulo.core.master.thrift.TabletServerStatus; import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.security.SystemPermission; -import org.apache.accumulo.core.security.TablePermission; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.SecurityUtil; import org.apache.accumulo.core.security.thrift.SecurityErrorCode; +import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper; import org.apache.accumulo.core.security.thrift.ThriftSecurityException; +import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper; import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction; import org.apache.accumulo.core.tabletserver.thrift.ActiveScan; import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException; @@ -157,10 +157,9 @@ import org.apache.accumulo.server.master import org.apache.accumulo.server.metrics.AbstractMetricsImpl; import org.apache.accumulo.server.problems.ProblemReport; import org.apache.accumulo.server.problems.ProblemReports; -import org.apache.accumulo.server.security.Authenticator; +import org.apache.accumulo.server.security.AuditedSecurityOperation; import org.apache.accumulo.server.security.SecurityConstants; -import org.apache.accumulo.server.security.SecurityUtil; -import org.apache.accumulo.server.security.ZKAuthenticator; +import org.apache.accumulo.server.security.SecurityOperation; import org.apache.accumulo.server.tabletserver.Compactor.CompactionInfo; import org.apache.accumulo.server.tabletserver.Tablet.CommitSession; import org.apache.accumulo.server.tabletserver.Tablet.KVEntry; @@ -580,13 +579,13 @@ public class TabletServer extends Abstra static class TservConstraintEnv implements Environment { - private AuthInfo credentials; - private Authenticator authenticator; + private InstanceTokenWrapper credentials; + private SecurityOperation security; private Authorizations auths; private KeyExtent ke; - TservConstraintEnv(Authenticator authenticator, AuthInfo credentials) { - this.authenticator = authenticator; + TservConstraintEnv(SecurityOperation secOp, InstanceTokenWrapper credentials) { + this.security = secOp; this.credentials = credentials; } @@ -601,15 +600,15 @@ public class TabletServer extends Abstra @Override public String getUser() { - return credentials.user; + return credentials.getPrincipal(); } @Override public Authorizations getAuthorizations() { if (auths == null) try { - this.auths = authenticator.getUserAuthorizations(credentials, getUser()); - } catch (AccumuloSecurityException e) { + this.auths = security.getUserAuthorizations(credentials); + } catch (ThriftSecurityException e) { throw new RuntimeException(e); } return auths; @@ -719,9 +718,9 @@ public class TabletServer extends Abstra public Tablet currentTablet; public MapCounter successfulCommits = new MapCounter(); Map failures = new HashMap(); - HashMap authFailures = new HashMap(); + HashMap authFailures = new HashMap(); public Violations violations; - public AuthInfo credentials; + public InstanceTokenWrapper credentials; public long totalUpdates = 0; public long flushTime = 0; Stat prepareTimes = new Stat(); @@ -878,15 +877,11 @@ public class TabletServer extends Abstra } @Override - public List bulkImport(TInfo tinfo, AuthInfo credentials, long tid, Map> files, boolean setTime) + public List bulkImport(TInfo tinfo, ThriftInstanceTokenWrapper titw, long tid, Map> files, boolean setTime) throws ThriftSecurityException { - - try { - if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); - } + InstanceTokenWrapper credentials = new InstanceTokenWrapper(titw); + if (!security.canPerformSystemActions(credentials)) + throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); List failures = new ArrayList(); @@ -1100,23 +1095,19 @@ public class TabletServer extends Abstra } @Override - public InitialScan startScan(TInfo tinfo, AuthInfo credentials, TKeyExtent textent, TRange range, List columns, int batchSize, + public InitialScan startScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent textent, TRange range, List columns, int batchSize, List ssiList, Map> ssio, List authorizations, boolean waitForWrites, boolean isolated) throws NotServingTabletException, ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException { Authorizations userauths = null; - - try { - if (!authenticator.hasTablePermission(credentials, credentials.user, new String(textent.getTable()), TablePermission.READ)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - - userauths = authenticator.getUserAuthorizations(credentials, credentials.user); - for (ByteBuffer auth : authorizations) - if (!userauths.contains(ByteBufferUtil.toBytes(auth))) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.BAD_AUTHORIZATIONS); - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); - } + InstanceTokenWrapper itw = new InstanceTokenWrapper(credentials); + if (!security.canScan(itw, new String(textent.getTable()))) + throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); + + userauths = security.getUserAuthorizations(itw); + for (ByteBuffer auth : authorizations) + if (!userauths.contains(ByteBufferUtil.toBytes(auth))) + throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS); KeyExtent extent = new KeyExtent(textent); @@ -1138,7 +1129,7 @@ public class TabletServer extends Abstra throw new NotServingTabletException(textent); ScanSession scanSession = new ScanSession(); - scanSession.user = credentials.user; + scanSession.user = itw.getPrincipal(); scanSession.extent = new KeyExtent(extent); scanSession.columnSet = new HashSet(); scanSession.ssiList = ssiList; @@ -1256,28 +1247,25 @@ public class TabletServer extends Abstra } @Override - public InitialMultiScan startMultiScan(TInfo tinfo, AuthInfo credentials, Map> tbatch, List tcolumns, + public InitialMultiScan startMultiScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, Map> tbatch, List tcolumns, List ssiList, Map> ssio, List authorizations, boolean waitForWrites) throws ThriftSecurityException { // find all of the tables that need to be scanned HashSet tables = new HashSet(); for (TKeyExtent keyExtent : tbatch.keySet()) { tables.add(new String(keyExtent.getTable())); } + InstanceTokenWrapper itw = new InstanceTokenWrapper(credentials); // check if user has permission to the tables Authorizations userauths = null; - try { - for (String table : tables) - if (!authenticator.hasTablePermission(credentials, credentials.user, table, TablePermission.READ)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - - userauths = authenticator.getUserAuthorizations(credentials, credentials.user); - for (ByteBuffer auth : authorizations) - if (!userauths.contains(ByteBufferUtil.toBytes(auth))) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.BAD_AUTHORIZATIONS); - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); - } + for (String table : tables) + if (!security.canScan(itw, table)) + throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); + + userauths = security.getUserAuthorizations(itw); + for (ByteBuffer auth : authorizations) + if (!userauths.contains(ByteBufferUtil.toBytes(auth))) + throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS); KeyExtent threadPoolExtent = null; @@ -1298,7 +1286,7 @@ public class TabletServer extends Abstra writeTracker.waitForWrites(TabletType.type(batch.keySet())); MultiScanSession mss = new MultiScanSession(); - mss.user = credentials.user; + mss.user = itw.getPrincipal(); mss.queries = batch; mss.columnSet = new HashSet(tcolumns.size()); mss.ssiList = ssiList; @@ -1384,22 +1372,18 @@ public class TabletServer extends Abstra } @Override - public long startUpdate(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException { + public long startUpdate(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException { // Make sure user is real - try { - if (!authenticator.authenticateUser(credentials, credentials.user, credentials.password)) { - if (updateMetrics.isEnabled()) - updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0); - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.BAD_CREDENTIALS); - } - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); - } + InstanceTokenWrapper itw = new InstanceTokenWrapper(credentials); + + security.authenticateUser(itw, itw.getToken()); + if (updateMetrics.isEnabled()) + updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0); UpdateSession us = new UpdateSession(); us.violations = new Violations(); - us.credentials = credentials; - us.cenv = new TservConstraintEnv(authenticator, credentials); + us.credentials = itw; + us.cenv = new TservConstraintEnv(security, us.credentials); long sid = sessionManager.createSession(us, false); @@ -1419,7 +1403,7 @@ public class TabletServer extends Abstra // if user has no permission to write to this table, add it to // the failures list boolean sameTable = us.currentTablet != null && (us.currentTablet.getExtent().getTableId().equals(keyExtent.getTableId())); - if (sameTable || authenticator.hasTablePermission(SecurityConstants.getSystemCredentials(), us.credentials.user, keyExtent.getTableId().toString(), TablePermission.WRITE)) { + if (sameTable || security.canWrite(us.credentials, keyExtent.getTableId().toString())) { long t2 = System.currentTimeMillis(); us.authTimes.addStat(t2 - t1); us.currentTablet = onlineTablets.get(keyExtent); @@ -1433,7 +1417,7 @@ public class TabletServer extends Abstra updateMetrics.add(TabletServerUpdateMetrics.unknownTabletErrors, 0); } } else { - log.warn("Denying access to table " + keyExtent.getTableId() + " for user " + us.credentials.user); + log.warn("Denying access to table " + keyExtent.getTableId() + " for user " + us.credentials.getPrincipal()); long t2 = System.currentTimeMillis(); us.authTimes.addStat(t2 - t1); us.currentTablet = null; @@ -1442,12 +1426,12 @@ public class TabletServer extends Abstra updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0); return; } - } catch (AccumuloSecurityException e) { - log.error("Denying permission to check user " + us.credentials.user + " with user " + e.getUser(), e); + } catch (ThriftSecurityException e) { + log.error("Denying permission to check user " + us.credentials.getPrincipal() + " with user " + e.getUser(), e); long t2 = System.currentTimeMillis(); us.authTimes.addStat(t2 - t1); us.currentTablet = null; - us.authFailures.put(keyExtent, e.getErrorCode()); + us.authFailures.put(keyExtent, e.getCode()); if (updateMetrics.isEnabled()) updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0); return; @@ -1657,19 +1641,16 @@ public class TabletServer extends Abstra log.debug(String.format("Authentication Failures: %d, first %s", us.authFailures.size(), first.toString())); } - return new UpdateErrors(Translator.translate(us.failures, Translator.KET), Translator.translate(violations, Translator.CVST), Translator.translate(us.authFailures, Translator.KET)); + return new UpdateErrors(Translator.translate(us.failures, Translator.KET), Translator.translate(violations, Translator.CVST), Translator.translate( + us.authFailures, Translator.KET)); } @Override - public void update(TInfo tinfo, AuthInfo credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException, + public void update(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException, ConstraintViolationException, ThriftSecurityException { - try { - if (!authenticator.hasTablePermission(credentials, credentials.user, new String(tkeyExtent.getTable()), TablePermission.WRITE)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); - } - + InstanceTokenWrapper itw = new InstanceTokenWrapper(credentials); + if (!security.canWrite(itw, new String(tkeyExtent.getTable()))) + throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); KeyExtent keyExtent = new KeyExtent(tkeyExtent); Tablet tablet = onlineTablets.get(new KeyExtent(keyExtent)); if (tablet == null) { @@ -1686,7 +1667,7 @@ public class TabletServer extends Abstra List mutations = Collections.singletonList(mutation); Span prep = Trace.start("prep"); - CommitSession cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(authenticator, credentials), mutations); + CommitSession cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(security, itw), mutations); prep.stop(); if (cs == null) { throw new NotServingTabletException(tkeyExtent); @@ -1714,17 +1695,13 @@ public class TabletServer extends Abstra } @Override - public void splitTablet(TInfo tinfo, AuthInfo credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint) throws NotServingTabletException, - ThriftSecurityException { + public void splitTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent tkeyExtent, ByteBuffer splitPoint) + throws NotServingTabletException, ThriftSecurityException { + InstanceTokenWrapper itw = new InstanceTokenWrapper(credentials); + String tableId = new String(ByteBufferUtil.toBytes(tkeyExtent.table)); - try { - if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.ALTER_TABLE) - && !authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM) - && !authenticator.hasTablePermission(credentials, credentials.user, tableId, TablePermission.ALTER_TABLE)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); - } + if (!security.canSplitTablet(itw, tableId)) + throw new ThriftSecurityException(itw.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); KeyExtent keyExtent = new KeyExtent(tkeyExtent); @@ -1746,12 +1723,12 @@ public class TabletServer extends Abstra } @Override - public TabletServerStatus getTabletServerStatus(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public TabletServerStatus getTabletServerStatus(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { return getStats(sessionManager.getActiveScansPerTable()); } @Override - public List getTabletStats(TInfo tinfo, AuthInfo credentials, String tableId) throws ThriftSecurityException, TException { + public List getTabletStats(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String tableId) throws ThriftSecurityException, TException { TreeMap onlineTabletsCopy; synchronized (onlineTablets) { onlineTabletsCopy = new TreeMap(onlineTablets); @@ -1777,22 +1754,23 @@ public class TabletServer extends Abstra private ZooCache masterLockCache = new ZooCache(); - private void checkPermission(AuthInfo credentials, String lock, boolean requiresSystemPermission, final String request) throws ThriftSecurityException { + private void checkPermission(InstanceTokenWrapper credentials, String lock, boolean requiresSystemPermission, final String request) + throws ThriftSecurityException { if (requiresSystemPermission) { boolean fatal = false; try { - log.debug("Got " + request + " message from user: " + credentials.user); - if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) { - log.warn("Got " + request + " message from user: " + credentials.user); - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); + log.debug("Got " + request + " message from user: " + credentials.getPrincipal()); + if (!security.canPerformSystemActions(credentials)) { + log.warn("Got " + request + " message from user: " + credentials.getPrincipal()); + throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED); } - } catch (AccumuloSecurityException e) { + } catch (ThriftSecurityException e) { log.warn("Got " + request + " message from unauthenticatable user: " + e.getUser()); if (e.getUser().equals(SecurityConstants.SYSTEM_USERNAME)) { log.fatal("Got message from a service with a mismatched configuration. Please ensure a compatible configuration.", e); fatal = true; } - throw e.asThriftException(); + throw e; } finally { if (fatal) { Halt.halt(1, new Runnable() { @@ -1820,27 +1798,31 @@ public class TabletServer extends Abstra }); } - ZooUtil.LockID lid = new ZooUtil.LockID(ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK, lock); - - try { - if (!ZooLock.isLockHeld(masterLockCache, lid)) { - // maybe the cache is out of date and a new master holds the - // lock? - masterLockCache.clear(); + if (lock != null) { + ZooUtil.LockID lid = new ZooUtil.LockID(ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK, lock); + + try { if (!ZooLock.isLockHeld(masterLockCache, lid)) { - log.warn("Got " + request + " message from a master that does not hold the current lock " + lock); - throw new RuntimeException("bad master lock"); + // maybe the cache is out of date and a new master holds the + // lock? + masterLockCache.clear(); + if (!ZooLock.isLockHeld(masterLockCache, lid)) { + log.warn("Got " + request + " message from a master that does not hold the current lock " + lock); + throw new RuntimeException("bad master lock"); + } } + } catch (Exception e) { + throw new RuntimeException("bad master lock", e); } - } catch (Exception e) { - throw new RuntimeException("bad master lock", e); } } @Override - public void loadTablet(TInfo tinfo, AuthInfo credentials, String lock, final TKeyExtent textent) { + public void loadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, final TKeyExtent textent) { + InstanceTokenWrapper itw = new InstanceTokenWrapper(credentials); + try { - checkPermission(credentials, lock, true, "loadTablet"); + checkPermission(itw, lock, true, "loadTablet"); } catch (ThriftSecurityException e) { log.error(e, e); throw new RuntimeException(e); @@ -1904,9 +1886,9 @@ public class TabletServer extends Abstra } @Override - public void unloadTablet(TInfo tinfo, AuthInfo credentials, String lock, TKeyExtent textent, boolean save) { + public void unloadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent textent, boolean save) { try { - checkPermission(credentials, lock, true, "unloadTablet"); + checkPermission(new InstanceTokenWrapper(credentials), lock, true, "unloadTablet"); } catch (ThriftSecurityException e) { log.error(e, e); throw new RuntimeException(e); @@ -1918,9 +1900,9 @@ public class TabletServer extends Abstra } @Override - public void flush(TInfo tinfo, AuthInfo credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) { + public void flush(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) { try { - checkPermission(credentials, lock, true, "flush"); + checkPermission(new InstanceTokenWrapper(credentials), lock, true, "flush"); } catch (ThriftSecurityException e) { log.error(e, e); throw new RuntimeException(e); @@ -1955,9 +1937,9 @@ public class TabletServer extends Abstra } @Override - public void flushTablet(TInfo tinfo, AuthInfo credentials, String lock, TKeyExtent textent) throws TException { + public void flushTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent textent) throws TException { try { - checkPermission(credentials, lock, true, "flushTablet"); + checkPermission(new InstanceTokenWrapper(credentials), lock, true, "flushTablet"); } catch (ThriftSecurityException e) { log.error(e, e); throw new RuntimeException(e); @@ -1974,9 +1956,9 @@ public class TabletServer extends Abstra } @Override - public void halt(TInfo tinfo, AuthInfo credentials, String lock) throws ThriftSecurityException { + public void halt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) throws ThriftSecurityException { - checkPermission(credentials, lock, true, "halt"); + checkPermission(new InstanceTokenWrapper(credentials), lock, true, "halt"); Halt.halt(0, new Runnable() { @Override @@ -1994,7 +1976,7 @@ public class TabletServer extends Abstra } @Override - public void fastHalt(TInfo info, AuthInfo credentials, String lock) { + public void fastHalt(TInfo info, ThriftInstanceTokenWrapper credentials, String lock) { try { halt(info, credentials, lock); } catch (Exception e) { @@ -2003,31 +1985,31 @@ public class TabletServer extends Abstra } @Override - public TabletStats getHistoricalStats(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public TabletStats getHistoricalStats(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { return statsKeeper.getTabletStats(); } @Override - public List getActiveScans(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { - + public List getActiveScans(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { try { - if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); + checkPermission(new InstanceTokenWrapper(credentials), null, true, "getScans"); + } catch (ThriftSecurityException e) { + log.error(e, e); + throw new RuntimeException(e); } return sessionManager.getActiveScans(); } @Override - public void chop(TInfo tinfo, AuthInfo credentials, String lock, TKeyExtent textent) throws TException { + public void chop(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent textent) throws TException { try { - if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - } catch (Exception e) { + checkPermission(new InstanceTokenWrapper(credentials), lock, true, "chop"); + } catch (ThriftSecurityException e) { + log.error(e, e); throw new RuntimeException(e); } + KeyExtent ke = new KeyExtent(textent); Tablet tablet = onlineTablets.get(ke); @@ -2037,11 +2019,12 @@ public class TabletServer extends Abstra } @Override - public void compact(TInfo tinfo, AuthInfo credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException { + public void compact(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) + throws TException { try { - if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - } catch (Exception e) { + checkPermission(new InstanceTokenWrapper(credentials), lock, true, "compact"); + } catch (ThriftSecurityException e) { + log.error(e, e); throw new RuntimeException(e); } @@ -2075,10 +2058,10 @@ public class TabletServer extends Abstra * (non-Javadoc) * * @see org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface#removeLogs(org.apache.accumulo.cloudtrace.thrift.TInfo, - * org.apache.accumulo.core.security.thrift.AuthInfo, java.util.List) + * org.apache.accumulo.core.security.thrift.InstanceTokenWrapper, java.util.List) */ @Override - public void removeLogs(TInfo tinfo, AuthInfo credentials, List filenames) throws TException { + public void removeLogs(TInfo tinfo, ThriftInstanceTokenWrapper credentials, List filenames) throws TException { String myname = getClientAddressString(); myname = myname.replace(':', '+'); Path logDir = new Path(Constants.getWalDirectory(acuConf), myname); @@ -2125,12 +2108,12 @@ public class TabletServer extends Abstra } @Override - public List getActiveCompactions(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public List getActiveCompactions(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { try { - if (!authenticator.hasSystemPermission(credentials, credentials.user, SystemPermission.SYSTEM)) - throw new ThriftSecurityException(credentials.user, SecurityErrorCode.PERMISSION_DENIED); - } catch (AccumuloSecurityException e) { - throw e.asThriftException(); + checkPermission(new InstanceTokenWrapper(credentials), null, true, "getActiveCompactions"); + } catch (ThriftSecurityException e) { + log.error(e, e); + throw new RuntimeException(e); } List compactions = Compactor.getRunningCompactions(); @@ -2183,20 +2166,20 @@ public class TabletServer extends Abstra synchronized (onlineTablets) { copyOnlineTablets.putAll(onlineTablets); // avoid - // concurrent - // modification + // concurrent + // modification } int numMajorCompactionsInProgress = 0; Iterator> iter = copyOnlineTablets.entrySet().iterator(); while (iter.hasNext() && !majorCompactorDisabled) { // bail - // early - // now - // if - // we're - // shutting - // down + // early + // now + // if + // we're + // shutting + // down Entry entry = iter.next(); @@ -2616,7 +2599,7 @@ public class TabletServer extends Abstra private InetSocketAddress clientAddress; private TabletServerResourceManager resourceManager; - private Authenticator authenticator; + private SecurityOperation security; private volatile boolean majorCompactorDisabled = false; private volatile boolean shutdownComplete = false; @@ -2820,7 +2803,7 @@ public class TabletServer extends Abstra while (!serverStopRequested && mm != null && client != null && client.getOutputProtocol() != null && client.getOutputProtocol().getTransport() != null && client.getOutputProtocol().getTransport().isOpen()) { try { - mm.send(SecurityConstants.getSystemCredentials(), getClientAddressString(), iface); + mm.send(SecurityConstants.getThriftSystemCredentials(), getClientAddressString(), iface); mm = null; } catch (TException ex) { log.warn("Error sending message: queuing message again"); @@ -3061,7 +3044,7 @@ public class TabletServer extends Abstra public void config(String hostname) { log.info("Tablet server starting on " + hostname); - authenticator = ZKAuthenticator.getInstance(); + security = AuditedSecurityOperation.getInstance(); clientAddress = new InetSocketAddress(hostname, 0); logger = new TabletServerLogger(this, getSystemConfiguration().getMemoryInBytes(Property.TSERV_WALOG_MAX_SIZE)); @@ -3088,7 +3071,7 @@ public class TabletServer extends Abstra String uris = null; boolean preDelegate = true; - + Iterator> iter = getSystemConfiguration().iterator(); while (iter.hasNext()) { Entry entry = iter.next(); @@ -3103,7 +3086,7 @@ public class TabletServer extends Abstra if (uris != null) return new ContextConfig(uris, preDelegate); - + return null; } }); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/MasterMessage.java Fri Jan 25 07:04:25 2013 @@ -17,12 +17,12 @@ package org.apache.accumulo.server.tabletserver.mastermessage; import org.apache.accumulo.core.master.thrift.MasterClientService; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper; import org.apache.accumulo.core.security.thrift.ThriftSecurityException; import org.apache.thrift.TException; public interface MasterMessage { - void send(AuthInfo info, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException; + void send(ThriftInstanceTokenWrapper info, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException; } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/SplitReportMessage.java Fri Jan 25 07:04:25 2013 @@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.i import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.master.thrift.MasterClientService; import org.apache.accumulo.core.master.thrift.TabletSplit; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper; import org.apache.accumulo.core.security.thrift.ThriftSecurityException; import org.apache.hadoop.io.Text; import org.apache.thrift.TException; @@ -45,7 +45,7 @@ public class SplitReportMessage implemen extents.put(ne2, np2); } - public void send(AuthInfo credentials, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException { + public void send(ThriftInstanceTokenWrapper credentials, String serverName, MasterClientService.Iface client) throws TException, ThriftSecurityException { TabletSplit split = new TabletSplit(); split.oldTablet = old_extent.toThrift(); split.newTablets = Translator.translate(extents.keySet(), Translator.KET); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/mastermessage/TabletStatusMessage.java Fri Jan 25 07:04:25 2013 @@ -20,7 +20,7 @@ import org.apache.accumulo.cloudtrace.in import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.master.thrift.TabletLoadState; import org.apache.accumulo.core.master.thrift.MasterClientService.Iface; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper; import org.apache.accumulo.core.security.thrift.ThriftSecurityException; import org.apache.thrift.TException; @@ -34,7 +34,7 @@ public class TabletStatusMessage impleme this.status = status; } - public void send(AuthInfo auth, String serverName, Iface client) throws TException, ThriftSecurityException { + public void send(ThriftInstanceTokenWrapper auth, String serverName, Iface client) throws TException, ThriftSecurityException { client.reportTabletStatus(Tracer.traceInfo(), auth, serverName, status, extent.toThrift()); } } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/GetMasterStats.java Fri Jan 25 07:04:25 2013 @@ -44,7 +44,7 @@ public class GetMasterStats { MasterMonitorInfo stats = null; try { client = MasterClient.getConnectionWithRetry(HdfsZooInstance.getInstance()); - stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials()); + stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getThriftSystemCredentials()); } finally { if (client != null) MasterClient.close(client); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/QueryMetadataTable.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/QueryMetadataTable.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/QueryMetadataTable.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/QueryMetadataTable.java Fri Jan 25 07:04:25 2013 @@ -25,7 +25,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.server.cli.ClientOpts; import org.apache.accumulo.core.cli.ScannerOpts; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; @@ -36,14 +35,15 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.tokens.UserPassToken; +import org.apache.accumulo.server.cli.ClientOpts; import org.apache.accumulo.server.client.HdfsZooInstance; import org.apache.hadoop.io.Text; import com.beust.jcommander.Parameter; public class QueryMetadataTable { - private static AuthInfo credentials; + private static UserPassToken credentials; static String location; @@ -58,7 +58,7 @@ public class QueryMetadataTable { try { KeyExtent extent = new KeyExtent(row, (Text) null); - Connector connector = HdfsZooInstance.getInstance().getConnector(credentials.user, credentials.password); + Connector connector = HdfsZooInstance.getInstance().getConnector(credentials); Scanner mdScanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS); Text row = extent.getMetadataEntry(); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/WrongTabletTest.java Fri Jan 25 07:04:25 2013 @@ -44,7 +44,7 @@ public class WrongTabletTest { Mutation mutation = new Mutation(new Text("row_0003750001")); mutation.putDelete(new Text("colf"), new Text("colq")); - client.update(Tracer.traceInfo(), opts.getAuthInfo(), new KeyExtent(new Text("!!"), null, new Text("row_0003750000")).toThrift(), mutation.toThrift()); + client.update(Tracer.traceInfo(), opts.getWrappedToken().toThrift(), new KeyExtent(new Text("!!"), null, new Text("row_0003750000")).toThrift(), mutation.toThrift()); } catch (Exception e) { throw new RuntimeException(e); } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/continuous/ContinuousStatsCollector.java Fri Jan 25 07:04:25 2013 @@ -126,7 +126,7 @@ public class ContinuousStatsCollector { MasterClientService.Iface client = null; try { client = MasterClient.getConnectionWithRetry(opts.getInstance()); - MasterMonitorInfo stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials()); + MasterMonitorInfo stats = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getThriftSystemCredentials()); TableInfo all = new TableInfo(); Map tableSummaries = new HashMap(); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/FunctionalTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/FunctionalTest.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/FunctionalTest.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/FunctionalTest.java Fri Jan 25 07:04:25 2013 @@ -16,7 +16,6 @@ */ package org.apache.accumulo.server.test.functional; -import java.nio.ByteBuffer; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -28,7 +27,6 @@ import java.util.TreeMap; import java.util.TreeSet; import org.apache.accumulo.core.Constants; -import org.apache.accumulo.server.cli.ClientOpts; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Connector; @@ -41,7 +39,10 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.tokens.AccumuloToken; +import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper; +import org.apache.accumulo.core.security.tokens.UserPassToken; +import org.apache.accumulo.server.cli.ClientOpts; import org.apache.accumulo.server.conf.ServerConfiguration; import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader; import org.apache.hadoop.fs.FileSystem; @@ -105,28 +106,19 @@ public abstract class FunctionalTest { } - private String username = ""; - private String password = ""; + private AccumuloToken token = new UserPassToken("", ""); private String instanceName = ""; - protected void setUsername(String username) { - this.username = username; - } - - protected String getUsername() { - return username; - } - - protected void setPassword(String password) { - this.password = password; + protected void setToken(AccumuloToken token) { + this.token = token; } - protected String getPassword() { - return password; + protected AccumuloToken getToken() { + return token; } protected Connector getConnector() throws AccumuloException, AccumuloSecurityException { - return getInstance().getConnector(username, password.getBytes()); + return getInstance().getConnector(getToken()); } protected Instance getInstance() { @@ -141,8 +133,8 @@ public abstract class FunctionalTest { return instanceName; } - protected AuthInfo getCredentials() { - return new AuthInfo(getUsername(), ByteBuffer.wrap(getPassword().getBytes()), getInstance().getInstanceID()); + protected InstanceTokenWrapper getCredentials() { + return new InstanceTokenWrapper(getToken(), getInstance().getInstanceID()); } public abstract Map getInitialConfig(); @@ -257,8 +249,7 @@ public abstract class FunctionalTest { FunctionalTest fTest = testClass.newInstance(); //fTest.setMaster(master); - fTest.setUsername(opts.user); - fTest.setPassword(new String(opts.getPassword())); + fTest.setToken(opts.getAccumuloToken()); fTest.setInstanceName(opts.instance); if (opts.opt.equals("getConfig")) { Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/PermissionsTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/PermissionsTest.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/PermissionsTest.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/PermissionsTest.java Fri Jan 25 07:04:25 2013 @@ -46,6 +46,7 @@ import org.apache.accumulo.core.security import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.core.security.thrift.SecurityErrorCode; +import org.apache.accumulo.core.security.tokens.UserPassToken; import org.apache.accumulo.server.conf.ServerConfiguration; import org.apache.hadoop.io.Text; import org.apache.log4j.Logger; @@ -53,6 +54,7 @@ import org.apache.log4j.Logger; public class PermissionsTest { private static final String TEST_USER = "test_user"; private static final String TEST_PASS = "test_password"; + private static final UserPassToken TEST_TOKEN = new UserPassToken(TEST_USER, TEST_PASS); public static class SystemPermissionsTest extends FunctionalTest { private static final Logger log = Logger.getLogger(SystemPermissionsTest.class); @@ -76,7 +78,7 @@ public class PermissionsTest { verifyHasOnlyTheseSystemPermissions(getConnector(), getConnector().whoami(), SystemPermission.values()); // create the test user - getConnector().securityOperations().createUser(TEST_USER, TEST_PASS.getBytes()); + getConnector().securityOperations().createUser(TEST_TOKEN); Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS.getBytes()); verifyHasNoSystemPermissions(getConnector(), TEST_USER, SystemPermission.values()); @@ -168,7 +170,7 @@ public class PermissionsTest { case CREATE_USER: user = "__CREATE_USER_WITHOUT_PERM_TEST__"; try { - test_user_conn.securityOperations().createUser(user, password.getBytes()); + test_user_conn.securityOperations().createUser(TEST_TOKEN); throw new IllegalStateException("Should NOT be able to create a user"); } catch (AccumuloSecurityException e) { if (e.getErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.securityOperations().authenticateUser(user, password.getBytes())) @@ -177,7 +179,7 @@ public class PermissionsTest { break; case DROP_USER: user = "__DROP_USER_WITHOUT_PERM_TEST__"; - root_conn.securityOperations().createUser(user, password.getBytes()); + root_conn.securityOperations().createUser(TEST_TOKEN); try { test_user_conn.securityOperations().dropUser(user); throw new IllegalStateException("Should NOT be able to delete a user"); @@ -188,7 +190,7 @@ public class PermissionsTest { break; case ALTER_USER: user = "__ALTER_USER_WITHOUT_PERM_TEST__"; - root_conn.securityOperations().createUser(user, password.getBytes()); + root_conn.securityOperations().createUser(TEST_TOKEN); try { test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B")); throw new IllegalStateException("Should NOT be able to alter a user"); @@ -235,6 +237,13 @@ public class PermissionsTest { if (!ServerConfiguration.getTableConfiguration(instance, tableId).get(Property.TABLE_BLOOM_ERRORRATE).equals("003.14159%")) throw new IllegalStateException("Should be able to set a table property"); test_user_conn.tableOperations().removeProperty(tableName, Property.TABLE_BLOOM_ERRORRATE.getKey()); + // Surprised we got away without pausing before. + try { + Thread.sleep(100); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } if (ServerConfiguration.getTableConfiguration(instance, tableId).get(Property.TABLE_BLOOM_ERRORRATE).equals("003.14159%")) throw new IllegalStateException("Should be able to remove a table property"); test_user_conn.tableOperations().rename(tableName, table2); @@ -243,20 +252,20 @@ public class PermissionsTest { break; case CREATE_USER: user = "__CREATE_USER_WITH_PERM_TEST__"; - test_user_conn.securityOperations().createUser(user, password.getBytes()); + test_user_conn.securityOperations().createUser(TEST_TOKEN); if (!root_conn.securityOperations().authenticateUser(user, password.getBytes())) throw new IllegalStateException("Should be able to create a user"); break; case DROP_USER: user = "__DROP_USER_WITH_PERM_TEST__"; - root_conn.securityOperations().createUser(user, password.getBytes()); + root_conn.securityOperations().createUser(TEST_TOKEN); test_user_conn.securityOperations().dropUser(user); if (root_conn.securityOperations().authenticateUser(user, password.getBytes())) throw new IllegalStateException("Should be able to delete a user"); break; case ALTER_USER: user = "__ALTER_USER_WITH_PERM_TEST__"; - root_conn.securityOperations().createUser(user, password.getBytes()); + root_conn.securityOperations().createUser(TEST_TOKEN); test_user_conn.securityOperations().changeUserAuthorizations(user, new Authorizations("A", "B")); if (root_conn.securityOperations().getUserAuthorizations(user).isEmpty()) throw new IllegalStateException("Should be able to alter a user"); @@ -297,6 +306,12 @@ public class PermissionsTest { private static final Logger log = Logger.getLogger(SystemPermissionsTest.class); private static final String TEST_TABLE = "__TABLE_PERMISSION_TEST__"; + public static void main(String args[]) throws Exception + { + FunctionalTest.main(args); + new TablePermissionsTest().run(); + } + @Override public void cleanup() throws Exception {} @@ -313,7 +328,7 @@ public class PermissionsTest { @Override public void run() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, MutationsRejectedException { // create the test user - getConnector().securityOperations().createUser(TEST_USER, TEST_PASS.getBytes()); + getConnector().securityOperations().createUser(TEST_TOKEN); Connector test_user_conn = getInstance().getConnector(TEST_USER, TEST_PASS.getBytes()); // check for read-only access to metadata table Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/VisibilityTest.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/VisibilityTest.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/VisibilityTest.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/VisibilityTest.java Fri Jan 25 07:04:25 2013 @@ -205,7 +205,7 @@ public class VisibilityTest extends Func private void queryData(Set allAuths, Set userAuths, Map,Set> expected) throws Exception { - getConnector().securityOperations().changeUserAuthorizations(getUsername(), new Authorizations(nbas(userAuths))); + getConnector().securityOperations().changeUserAuthorizations(getToken().getPrincipal(), new Authorizations(nbas(userAuths))); ArrayList> combos = new ArrayList>(); uniqueCombos(combos, nss(), allAuths); @@ -232,7 +232,7 @@ public class VisibilityTest extends Func Scanner scanner; // should return no records - getConnector().securityOperations().changeUserAuthorizations(getUsername(), new Authorizations("BASE", "DEFLABEL")); + getConnector().securityOperations().changeUserAuthorizations(getToken().getPrincipal(), new Authorizations("BASE", "DEFLABEL")); scanner = getConnector().createScanner("vt2", new Authorizations()); verifyDefault(scanner, 0); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/functional/ZombieTServer.java Fri Jan 25 07:04:25 2013 @@ -27,7 +27,7 @@ import org.apache.accumulo.core.Constant import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.master.thrift.TableInfo; import org.apache.accumulo.core.master.thrift.TabletServerStatus; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper; import org.apache.accumulo.core.security.thrift.ThriftSecurityException; import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface; import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor; @@ -64,13 +64,13 @@ public class ZombieTServer { } @Override - synchronized public void fastHalt(TInfo tinfo, AuthInfo credentials, String lock) { + synchronized public void fastHalt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) { halted = true; notifyAll(); } @Override - public TabletServerStatus getTabletServerStatus(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public TabletServerStatus getTabletServerStatus(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { synchronized (this) { if (statusCount++ < 1) { TabletServerStatus result = new TabletServerStatus(); @@ -83,7 +83,7 @@ public class ZombieTServer { } @Override - synchronized public void halt(TInfo tinfo, AuthInfo credentials, String lock) throws ThriftSecurityException, TException { + synchronized public void halt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) throws ThriftSecurityException, TException { halted = true; notifyAll(); } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java Fri Jan 25 07:04:25 2013 @@ -18,7 +18,6 @@ package org.apache.accumulo.server.test. import java.io.IOException; import java.net.InetAddress; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -64,7 +63,8 @@ import org.apache.accumulo.core.iterator import org.apache.accumulo.core.iterators.system.MultiIterator; import org.apache.accumulo.core.iterators.system.VisibilityFilter; import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.tokens.AccumuloToken; +import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper; import org.apache.accumulo.core.util.AddressUtil; import org.apache.accumulo.core.util.CachedConfiguration; import org.apache.accumulo.core.util.MetadataTable; @@ -120,7 +120,7 @@ public class CollectTabletStats { } Map locations = new HashMap(); - List candidates = findTablets(!opts.selectFarTablets, opts.user, opts.getPassword(), opts.tableName, instance, locations); + List candidates = findTablets(!opts.selectFarTablets, opts.getAccumuloToken(), opts.tableName, instance, locations); if (candidates.size() < opts.numThreads) { System.err.println("ERROR : Unable to find " + opts.numThreads + " " + (opts.selectFarTablets ? "far" : "local") + " tablets"); @@ -132,7 +132,7 @@ public class CollectTabletStats { Map> tabletFiles = new HashMap>(); for (KeyExtent ke : tabletsToTest) { - List files = getTabletFiles(opts.user, opts.getPassword(), opts.getInstance(), tableId, ke); + List files = getTabletFiles(opts.getAccumuloToken(), opts.getInstance(), tableId, ke); tabletFiles.put(ke, files); } @@ -340,11 +340,11 @@ public class CollectTabletStats { } - private static List findTablets(boolean selectLocalTablets, String user, byte[] pass, String table, Instance zki, + private static List findTablets(boolean selectLocalTablets, AccumuloToken token, String table, Instance zki, Map locations) throws Exception { SortedSet tablets = new TreeSet(); - MetadataTable.getEntries(zki, new AuthInfo(user, ByteBuffer.wrap(pass), zki.getInstanceID()), table, false, locations, tablets); + MetadataTable.getEntries(zki, new InstanceTokenWrapper(token, zki.getInstanceID()), table, false, locations, tablets); InetAddress localaddress = InetAddress.getLocalHost(); @@ -375,11 +375,11 @@ public class CollectTabletStats { return tabletsToTest; } - private static List getTabletFiles(String user, byte[] pass, Instance zki, String tableId, KeyExtent ke) { + private static List getTabletFiles(AccumuloToken token, Instance zki, String tableId, KeyExtent ke) { List files = new ArrayList(); SortedMap tkv = new TreeMap(); - MetadataTable.getTabletAndPrevTabletKeyValues(zki, tkv, ke, null, new AuthInfo(user, ByteBuffer.wrap(pass), zki.getInstanceID())); + MetadataTable.getTabletAndPrevTabletKeyValues(zki, tkv, ke, null, new InstanceTokenWrapper(token, zki.getInstanceID())); Set> es = tkv.entrySet(); for (Entry entry : es) { Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/thrift/NullTserver.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/thrift/NullTserver.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/thrift/NullTserver.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/performance/thrift/NullTserver.java Fri Jan 25 07:04:25 2013 @@ -46,7 +46,7 @@ import org.apache.accumulo.core.data.thr import org.apache.accumulo.core.data.thrift.TRange; import org.apache.accumulo.core.data.thrift.UpdateErrors; import org.apache.accumulo.core.master.thrift.TabletServerStatus; -import org.apache.accumulo.core.security.thrift.AuthInfo; +import org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper; import org.apache.accumulo.core.security.thrift.SecurityErrorCode; import org.apache.accumulo.core.security.thrift.ThriftSecurityException; import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction; @@ -89,7 +89,7 @@ public class NullTserver { } @Override - public long startUpdate(TInfo tinfo, AuthInfo credentials) { + public long startUpdate(TInfo tinfo, ThriftInstanceTokenWrapper credentials) { return updateSession++; } @@ -102,7 +102,7 @@ public class NullTserver { } @Override - public List bulkImport(TInfo tinfo, AuthInfo credentials, long tid, Map> files, boolean setTime) { + public List bulkImport(TInfo tinfo, ThriftInstanceTokenWrapper credentials, long tid, Map> files, boolean setTime) { return null; } @@ -123,74 +123,74 @@ public class NullTserver { } @Override - public void splitTablet(TInfo tinfo, AuthInfo credentials, TKeyExtent extent, ByteBuffer splitPoint) { + public void splitTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent extent, ByteBuffer splitPoint) { } @Override - public InitialMultiScan startMultiScan(TInfo tinfo, AuthInfo credentials, Map> batch, List columns, + public InitialMultiScan startMultiScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, Map> batch, List columns, List ssiList, Map> ssio, List authorizations, boolean waitForWrites) { return null; } @Override - public InitialScan startScan(TInfo tinfo, AuthInfo credentials, TKeyExtent extent, TRange range, List columns, int batchSize, + public InitialScan startScan(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent extent, TRange range, List columns, int batchSize, List ssiList, Map> ssio, List authorizations, boolean waitForWrites, boolean isolated) { return null; } @Override - public void update(TInfo tinfo, AuthInfo credentials, TKeyExtent keyExtent, TMutation mutation) { + public void update(TInfo tinfo, ThriftInstanceTokenWrapper credentials, TKeyExtent keyExtent, TMutation mutation) { } @Override - public TabletServerStatus getTabletServerStatus(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public TabletServerStatus getTabletServerStatus(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { return null; } @Override - public List getTabletStats(TInfo tinfo, AuthInfo credentials, String tableId) throws ThriftSecurityException, TException { + public List getTabletStats(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String tableId) throws ThriftSecurityException, TException { return null; } @Override - public TabletStats getHistoricalStats(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public TabletStats getHistoricalStats(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { return null; } @Override - public void halt(TInfo tinfo, AuthInfo credentials, String lock) throws ThriftSecurityException, TException {} + public void halt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) throws ThriftSecurityException, TException {} @Override - public void fastHalt(TInfo tinfo, AuthInfo credentials, String lock) {} + public void fastHalt(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock) {} @Override - public void loadTablet(TInfo tinfo, AuthInfo credentials, String lock, TKeyExtent extent) throws TException {} + public void loadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent) throws TException {} @Override - public void unloadTablet(TInfo tinfo, AuthInfo credentials, String lock, TKeyExtent extent, boolean save) throws TException {} + public void unloadTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent, boolean save) throws TException {} @Override - public List getActiveScans(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public List getActiveScans(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { return new ArrayList(); } @Override - public void chop(TInfo tinfo, AuthInfo credentials, String lock, TKeyExtent extent) throws TException {} + public void chop(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent) throws TException {} @Override - public void flushTablet(TInfo tinfo, AuthInfo credentials, String lock, TKeyExtent extent) throws TException { + public void flushTablet(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, TKeyExtent extent) throws TException { } @Override - public void compact(TInfo tinfo, AuthInfo credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException { + public void compact(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException { } @Override - public void flush(TInfo tinfo, AuthInfo credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException { + public void flush(TInfo tinfo, ThriftInstanceTokenWrapper credentials, String lock, String tableId, ByteBuffer startRow, ByteBuffer endRow) throws TException { } @@ -198,14 +198,14 @@ public class NullTserver { * (non-Javadoc) * * @see org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface#removeLogs(org.apache.accumulo.cloudtrace.thrift.TInfo, - * org.apache.accumulo.core.security.thrift.AuthInfo, java.util.List) + * org.apache.accumulo.core.security.thrift.ThriftInstanceTokenWrapper, java.util.List) */ @Override - public void removeLogs(TInfo tinfo, AuthInfo credentials, List filenames) throws TException { + public void removeLogs(TInfo tinfo, ThriftInstanceTokenWrapper credentials, List filenames) throws TException { } @Override - public List getActiveCompactions(TInfo tinfo, AuthInfo credentials) throws ThriftSecurityException, TException { + public List getActiveCompactions(TInfo tinfo, ThriftInstanceTokenWrapper credentials) throws ThriftSecurityException, TException { return new ArrayList(); } } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/Module.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/Module.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/Module.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/Module.java Fri Jan 25 07:04:25 2013 @@ -162,7 +162,7 @@ public class Module extends Node { this.xmlFile = xmlFile; loadFromXml(); } - + @Override public void visit(State state, Properties props) throws Exception { int maxHops, maxSec; @@ -200,7 +200,7 @@ public class Module extends Node { initNode.visit(state, getProps(initNodeId)); if (test) stopTimer(initNode); - + state.visitedNode(); // update aliases Set aliases; @@ -256,6 +256,11 @@ public class Module extends Node { for (Entry entry : nodeProps.entrySet()) { log.debug(" " + entry.getKey() + ": " + entry.getValue()); } + log.debug("State information: "); + for (String key : new TreeSet(state.getMap().keySet())) { + Object value = state.getMap().get(key); + log.debug(" " + key + ": " + value + ' ' + (value != null && ! (value instanceof String)? ((value.getClass().equals(byte[].class))? new String((byte[]) value):value.getClass() + " - " + value):"")); + } throw new Exception("Error running node " + nextNodeId, e); } state.visitedNode(); @@ -279,7 +284,7 @@ public class Module extends Node { final int time = 5 * 1000 * 60; AtomicBoolean runningLong = new AtomicBoolean(false); long systemTime; - + /** * */ @@ -321,7 +326,7 @@ public class Module extends Node { if (runningLong.get()) log.warn("Node " + nextNode + ", which was running long, has now completed after " + (System.currentTimeMillis() - systemTime) / 1000.0 + " seconds"); } - + @Override public String toString() { return xmlFile.toString(); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/State.java Fri Jan 25 07:04:25 2013 @@ -18,15 +18,20 @@ package org.apache.accumulo.server.test. import java.io.File; import java.lang.management.ManagementFactory; +import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Properties; 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.BatchWriterConfig; import org.apache.accumulo.core.client.Connector; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.MultiTableBatchWriter; import org.apache.accumulo.core.client.ZooKeeperInstance; +import org.apache.accumulo.core.security.tokens.InstanceTokenWrapper; +import org.apache.accumulo.core.security.tokens.UserPassToken; import org.apache.accumulo.server.client.HdfsZooInstance; import org.apache.log4j.Logger; @@ -73,29 +78,38 @@ public class State { return stateMap.get(key); } + public HashMap getMap() { + return stateMap; + } + public String getString(String key) { return (String) stateMap.get(key); } - public Integer getInteger(String key) { - return (Integer) stateMap.get(key); + public Long getLong(String key) { + return (Long) stateMap.get(key); } public String getProperty(String key) { return props.getProperty(key); } - public Connector getConnector() throws Exception { + public Connector getConnector() throws AccumuloException, AccumuloSecurityException { if (connector == null) { String instance = props.getProperty("INSTANCE"); String zookeepers = props.getProperty("ZOOKEEPERS"); - String username = props.getProperty("USERNAME"); - String password = props.getProperty("PASSWORD"); - connector = new ZooKeeperInstance(instance, zookeepers).getConnector(username, password.getBytes()); + InstanceTokenWrapper auth = getAuthInfo(); + connector = new ZooKeeperInstance(instance, zookeepers).getConnector(auth); } return connector; } + public InstanceTokenWrapper getAuthInfo() { + String username = props.getProperty("USERNAME"); + String password = props.getProperty("PASSWORD"); + return new InstanceTokenWrapper(new UserPassToken(username, ByteBuffer.wrap(password.getBytes())), this.getInstance().getInstanceID()); + } + public Instance getInstance() { if (instance == null) { instance = HdfsZooInstance.getInstance(); Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/concurrent/CreateUser.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/concurrent/CreateUser.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/concurrent/CreateUser.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/concurrent/CreateUser.java Fri Jan 25 07:04:25 2013 @@ -22,6 +22,7 @@ import java.util.Random; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.security.tokens.UserPassToken; import org.apache.accumulo.server.test.randomwalk.State; import org.apache.accumulo.server.test.randomwalk.Test; @@ -39,7 +40,7 @@ public class CreateUser extends Test { try { log.debug("Creating user " + userName); - conn.securityOperations().createUser(userName, (userName + "pass").getBytes()); + conn.securityOperations().createUser(new UserPassToken(userName, (userName + "pass"))); } catch (AccumuloSecurityException ex) { log.debug("Create user failed " + ex.getCause()); } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Commit.java Fri Jan 25 07:04:25 2013 @@ -27,7 +27,7 @@ public class Commit extends Test { public void visit(State state, Properties props) throws Exception { state.getMultiTableBatchWriter().flush(); - log.debug("Committed " + state.getInteger("numWrites") + " writes. Total writes: " + state.getInteger("totalWrites")); + log.debug("Committed " + state.getLong("numWrites") + " writes. Total writes: " + state.getLong("totalWrites")); state.set("numWrites", new Integer(0)); } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/ImageFixture.java Fri Jan 25 07:04:25 2013 @@ -113,6 +113,6 @@ public class ImageFixture extends Fixtur conn.tableOperations().delete(imageTableName); conn.tableOperations().delete(indexTableName); - log.debug("Final total of writes: " + state.getInteger("totalWrites")); + log.debug("Final total of writes: " + state.getLong("totalWrites")); } } Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/image/Write.java Fri Jan 25 07:04:25 2013 @@ -69,8 +69,8 @@ public class Write extends Test { m.put(META_COLUMN_FAMILY, SHA1_COLUMN_QUALIFIER, new Value(hash)); // update write counts - state.set("numWrites", state.getInteger("numWrites") + 1); - Integer totalWrites = state.getInteger("totalWrites") + 1; + state.set("numWrites", state.getLong("numWrites") + 1); + Long totalWrites = state.getLong("totalWrites") + 1; state.set("totalWrites", totalWrites); // set count Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java?rev=1438354&r1=1438353&r2=1438354&view=diff ============================================================================== --- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java (original) +++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/test/randomwalk/multitable/Commit.java Fri Jan 25 07:04:25 2013 @@ -27,8 +27,8 @@ public class Commit extends Test { public void visit(State state, Properties props) throws Exception { state.getMultiTableBatchWriter().flush(); - Integer numWrites = state.getInteger("numWrites"); - Integer totalWrites = state.getInteger("totalWrites") + numWrites; + Long numWrites = state.getLong("numWrites"); + Long totalWrites = state.getLong("totalWrites") + numWrites; log.debug("Committed " + numWrites + " writes. Total writes: " + totalWrites);