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 6893ADD31 for ; Thu, 23 Aug 2012 12:39:19 +0000 (UTC) Received: (qmail 5762 invoked by uid 500); 23 Aug 2012 12:39:19 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 5613 invoked by uid 500); 23 Aug 2012 12:39:15 -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 5543 invoked by uid 99); 23 Aug 2012 12:39:13 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 23 Aug 2012 12:39:13 +0000 X-ASF-Spam-Status: No, hits=-1998.5 required=5.0 tests=ALL_TRUSTED,FILL_THIS_FORM_SHORT 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; Thu, 23 Aug 2012 12:39:10 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 1FE612388994 for ; Thu, 23 Aug 2012 12:38:27 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1376464 - in /accumulo/branches/ACCUMULO-722/1.5: ./ core/ core/src/main/java/org/apache/accumulo/core/ core/src/main/java/org/apache/accumulo/core/client/ core/src/main/java/org/apache/accumulo/core/client/impl/ core/src/main/java/org/apa... Date: Thu, 23 Aug 2012 12:38:26 -0000 To: commits@accumulo.apache.org From: ecn@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20120823123827.1FE612388994@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: ecn Date: Thu Aug 23 12:38:25 2012 New Revision: 1376464 URL: http://svn.apache.org/viewvc?rev=1376464&view=rev Log: ACCUMULO-722 fixed zookeeper delete/rename Modified: accumulo/branches/ACCUMULO-722/1.5/ (props changed) accumulo/branches/ACCUMULO-722/1.5/core/ (props changed) accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/Constants.java accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java accumulo/branches/ACCUMULO-722/1.5/pom.xml accumulo/branches/ACCUMULO-722/1.5/server/ (props changed) accumulo/branches/ACCUMULO-722/1.5/server/pom.xml accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoverLease.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/security/ZKAuthenticator.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/MinorCompactor.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/TabletServerLogger.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/Initialize.java accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java accumulo/branches/ACCUMULO-722/1.5/src/ (props changed) accumulo/branches/ACCUMULO-722/1.5/test/system/continuous/ (props changed) Propchange: accumulo/branches/ACCUMULO-722/1.5/ ------------------------------------------------------------------------------ Merged /accumulo/branches/ACCUMULO-722/1.4/src:r1370866-1371470 Merged /accumulo/branches/ACCUMULO-722/1.4:r1370866-1371470 Propchange: accumulo/branches/ACCUMULO-722/1.5/core/ ------------------------------------------------------------------------------ Merged /accumulo/branches/ACCUMULO-722/1.4/src/core:r1370866-1371470 Modified: accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/Constants.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/Constants.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/Constants.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/Constants.java Thu Aug 23 12:38:25 2012 @@ -37,7 +37,7 @@ public class Constants { public static final String ZINSTANCES = "/instances"; public static final String ZTABLES = "/tables"; - public static final byte[] ZTABLES_INITIAL_ID = new byte[] {'0'}; + public static final byte[] ZTABLES_INITIAL_ID = new byte[] {'3'}; public static final String ZTABLE_NAME = "/name"; public static final String ZTABLE_CONF = "/conf"; public static final String ZTABLE_STATE = "/state"; Modified: accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java Thu Aug 23 12:38:25 2012 @@ -18,6 +18,7 @@ package org.apache.accumulo.core.client; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.UUID; @@ -266,7 +267,7 @@ public class ZooKeeperInstance implement throw new RuntimeException("Accumulo found multiple possible instance ids in " + instanceDirectory); } else { String result = files[0].getPath().getName(); - return result; + return result; } } catch (IOException e) { throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory, e); Modified: accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java Thu Aug 23 12:38:25 2012 @@ -42,8 +42,11 @@ import org.apache.accumulo.core.security import org.apache.accumulo.core.security.thrift.AuthInfo; import org.apache.accumulo.core.security.thrift.SecurityErrorCode; import org.apache.accumulo.core.util.ArgumentChecker; +import org.apache.log4j.Logger; public class ConnectorImpl extends Connector { + static private final Logger log = Logger.getLogger(ConnectorImpl.class); + private Instance instance; private AuthInfo credentials; private SecurityOperations secops = null; @@ -73,15 +76,15 @@ public class ConnectorImpl extends Conne // hardcoded string for SYSTEM user since the definition is // in server code - if (!user.equals("!SYSTEM")) { - ServerClient.execute(instance, new ClientExec() { - @Override - public void execute(ClientService.Client iface) throws Exception { - if (!iface.authenticateUser(Tracer.traceInfo(), credentials, credentials.user, credentials.password)) - throw new AccumuloSecurityException("Authentication failed, access denied", SecurityErrorCode.BAD_CREDENTIALS); - } - }); - } +// if (!user.equals("!SYSTEM")) { +// ServerClient.execute(instance, new ClientExec() { +// @Override +// public void execute(ClientService.Client iface) throws Exception { +// if (!iface.authenticateUser(Tracer.traceInfo(), credentials, credentials.user, credentials.password)) +// throw new AccumuloSecurityException("Authentication failed, access denied", SecurityErrorCode.BAD_CREDENTIALS); +// } +// }); +// } } private String getTableId(String tableName) throws TableNotFoundException { Modified: accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java Thu Aug 23 12:38:25 2012 @@ -150,7 +150,7 @@ public class ServerClient { if (!opened) { if (!warnedAboutTServersBeingDown) { if (servers.isEmpty()) { - log.warn("There are no tablet servers: check that zookeeper and accumulo are running."); + log.warn("There are no tablet servers: check that zookeeper and accumulo are running.", new Throwable()); } else { log.warn("Failed to find an available server in the list of servers: " + servers); } Modified: accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java Thu Aug 23 12:38:25 2012 @@ -54,7 +54,7 @@ public class TabletServerBatchReader ext } private final int batchReaderInstance = getNextBatchReaderInstance(); - + public TabletServerBatchReader(Instance instance, AuthInfo credentials, String table, Authorizations authorizations, int numQueryThreads) { ArgumentChecker.notNull(instance, credentials, table, authorizations); this.instance = instance; Modified: accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java Thu Aug 23 12:38:25 2012 @@ -31,6 +31,7 @@ import java.lang.ref.WeakReference; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.Map.Entry; import java.util.Set; import java.util.SortedMap; @@ -50,6 +51,11 @@ import org.apache.hadoop.io.WritableComp public class KeyExtent implements WritableComparable { private static WeakHashMap> tableIds = new WeakHashMap>(); + static private final Set METASET = new HashSet(); + static { + for (String id : "!0,0,1,2".split(",")) + METASET.add(new Text(id)); + } private static Text dedupeTableId(Text tableId) { synchronized (tableIds) { @@ -774,7 +780,7 @@ public class KeyExtent implements Writab } public boolean isMeta() { - return getTableId().toString().equals(Constants.METADATA_TABLE_ID); + return METASET.contains(getTableId()); } public boolean isRootTablet() { Modified: accumulo/branches/ACCUMULO-722/1.5/pom.xml URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/pom.xml?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/pom.xml (original) +++ accumulo/branches/ACCUMULO-722/1.5/pom.xml Thu Aug 23 12:38:25 2012 @@ -143,7 +143,7 @@ ../lib - commons-collections,commons-configuration,commons-io,commons-lang,jline,log4j,libthrift,commons-jci-core,commons-jci-fam,commons-logging,commons-logging-api + commons-collections,commons-configuration,commons-io,commons-lang,jline,log4j,libthrift,commons-jci-core,commons-jci-fam,commons-logging,commons-logging-api,curator-framework,curator-client,guava true @@ -558,7 +558,7 @@ commons-configuration commons-configuration - 1.5 + 1.6 commons-cli @@ -569,12 +569,12 @@ commons-io commons-io - 1.4 + 2.1 commons-collections commons-collections - 3.2 + 3.2.1 org.apache.thrift @@ -599,7 +599,7 @@ commons-codec commons-codec - 1.3 + 1.4 org.slf4j @@ -621,6 +621,21 @@ commons-jci-fam 1.0 + + com.netflix.curator + curator-framework + 1.1.15 + + + com.netflix.curator + curator-client + 1.1.15 + + + com.google.guava + guava + 12.0.1 + Propchange: accumulo/branches/ACCUMULO-722/1.5/server/ ------------------------------------------------------------------------------ Merged /accumulo/branches/ACCUMULO-722/1.4/src/server:r1370866-1371470 Modified: accumulo/branches/ACCUMULO-722/1.5/server/pom.xml URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/pom.xml?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/pom.xml (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/pom.xml Thu Aug 23 12:38:25 2012 @@ -114,6 +114,18 @@ commons-lang commons-lang + + com.netflix.curator + curator-framework + + + com.netflix.curator + curator-client + + + com.google.guava + guava + Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoverLease.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoverLease.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoverLease.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/recovery/RecoverLease.java Thu Aug 23 12:38:25 2012 @@ -24,6 +24,7 @@ import org.apache.accumulo.fate.Repo; import org.apache.accumulo.server.master.Master; import org.apache.accumulo.server.master.tableOps.MasterRepo; import org.apache.accumulo.server.trace.TraceFileSystem; +import org.apache.commons.lang.NotImplementedException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; @@ -88,7 +89,8 @@ public class RecoverLease extends Master try { fs.append(source).close(); log.info("Recovered lease on " + source.toString() + " using append"); - + } catch (NotImplementedException ex) { + return new SubmitFileForRecovery(server, file); } catch (IOException ex) { log.error("error recovering lease using append", ex); } Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/master/state/tables/TableManager.java Thu Aug 23 12:38:25 2012 @@ -26,6 +26,8 @@ import java.util.Set; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.Instance; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.master.state.tables.TableState; import org.apache.accumulo.core.zookeeper.ZooUtil; import org.apache.accumulo.fate.zookeeper.IZooReaderWriter; @@ -60,7 +62,13 @@ public class TableManager { String zTablePath = Constants.ZROOT + "/" + instanceId + Constants.ZTABLES + "/" + tableId; IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance(); zoo.putPersistentData(zTablePath, new byte[0], existsPolicy); - zoo.putPersistentData(zTablePath + Constants.ZTABLE_CONF, new byte[0], existsPolicy); + String zTableConf = zTablePath + Constants.ZTABLE_CONF; + zoo.putPersistentData(zTableConf, new byte[0], existsPolicy); + String iter = Property.TABLE_ITERATOR_PREFIX.getKey(); + for (String when : "scan,minc,majc".split(",")) { + zoo.putPersistentData(zTableConf + "/" + iter + when + ".vers", ("10," + VersioningIterator.class.getName()).getBytes(), existsPolicy); + zoo.putPersistentData(zTableConf + "/" + iter + when + ".vers.opt.maxVersions", "1".getBytes(), existsPolicy); + } zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAME, tableName.getBytes(), existsPolicy); zoo.putPersistentData(zTablePath + Constants.ZTABLE_STATE, state.name().getBytes(), existsPolicy); zoo.putPersistentData(zTablePath + Constants.ZTABLE_FLUSH_ID, "0".getBytes(), existsPolicy); Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/monitor/servlets/DefaultServlet.java Thu Aug 23 12:38:25 2012 @@ -295,6 +295,8 @@ public class DefaultServlet extends Basi private void doAccumuloTable(StringBuilder sb) throws IOException { // Accumulo + if (true) + return ; Configuration conf = CachedConfiguration.getInstance(); FileSystem fs = TraceFileSystem.wrap(FileUtil.getFileSystem(conf, ServerConfiguration.getSiteConfiguration())); MasterMonitorInfo info = Monitor.getMmi(); @@ -334,37 +336,41 @@ public class DefaultServlet extends Basi private void doHdfsTable(StringBuilder sb) throws IOException { // HDFS Configuration conf = CachedConfiguration.getInstance(); - DistributedFileSystem fs = (DistributedFileSystem) FileSystem.get(conf); - String httpAddress = conf.get("dfs.http.address"); - String port = httpAddress.split(":")[1]; - String href = "http://" + fs.getUri().getHost() + ":" + port; - String liveUrl = href + "/dfsnodelist.jsp?whatNodes=LIVE"; - String deadUrl = href + "/dfsnodelist.jsp?whatNodes=DEAD"; - sb.append("\n"); - sb.append("\n"); - try { - boolean highlight = false; - tableRow(sb, (highlight = !highlight), "Unreplicated Capacity", bytes(fs.getRawCapacity())); - tableRow(sb, (highlight = !highlight), "% Used", NumberType.commas(fs.getRawUsed() * 100. / fs.getRawCapacity(), 0, 90, 0, 100) + "%"); - tableRow(sb, (highlight = !highlight), "Corrupt Blocks", NumberType.commas(fs.getCorruptBlocksCount(), 0, 0)); - DatanodeInfo[] liveNodes = fs.getClient().datanodeReport(FSConstants.DatanodeReportType.LIVE); - DatanodeInfo[] deadNodes = fs.getClient().datanodeReport(FSConstants.DatanodeReportType.DEAD); - tableRow(sb, (highlight = !highlight), "Live Data Nodes", NumberType.commas(liveNodes.length)); - tableRow(sb, (highlight = !highlight), "Dead Data Nodes", NumberType.commas(deadNodes.length)); - long count = 0; - for (DatanodeInfo stat : liveNodes) - count += stat.getXceiverCount(); - tableRow(sb, (highlight = !highlight), "Xceivers", NumberType.commas(count)); - } catch (RemoteException ex) { - sb.append("\n"); - } catch (Exception ex) { - sb.append("\n"); + FileSystem fs = FileSystem.get(conf); + if (fs instanceof DistributedFileSystem) { + DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get(conf); + String httpAddress = conf.get("dfs.http.address"); + String port = httpAddress.split(":")[1]; + String href = "http://" + dfs.getUri().getHost() + ":" + port; + String liveUrl = href + "/dfsnodelist.jsp?whatNodes=LIVE"; + String deadUrl = href + "/dfsnodelist.jsp?whatNodes=DEAD"; + sb.append("
NameNode
Permission Denied
Down
\n"); + sb.append("\n"); + try { + boolean highlight = false; + tableRow(sb, (highlight = !highlight), "Unreplicated Capacity", bytes(dfs.getRawCapacity())); + tableRow(sb, (highlight = !highlight), "% Used", NumberType.commas(dfs.getRawUsed() * 100. / dfs.getRawCapacity(), 0, 90, 0, 100) + "%"); + tableRow(sb, (highlight = !highlight), "Corrupt Blocks", NumberType.commas(dfs.getCorruptBlocksCount(), 0, 0)); + DatanodeInfo[] liveNodes = dfs.getClient().datanodeReport(FSConstants.DatanodeReportType.LIVE); + DatanodeInfo[] deadNodes = dfs.getClient().datanodeReport(FSConstants.DatanodeReportType.DEAD); + tableRow(sb, (highlight = !highlight), "Live Data Nodes", NumberType.commas(liveNodes.length)); + tableRow(sb, (highlight = !highlight), "Dead Data Nodes", NumberType.commas(deadNodes.length)); + long count = 0; + for (DatanodeInfo stat : liveNodes) + count += stat.getXceiverCount(); + tableRow(sb, (highlight = !highlight), "Xceivers", NumberType.commas(count)); + } catch (RemoteException ex) { + sb.append("\n"); + } catch (Exception ex) { + sb.append("\n"); + } } - sb.append("
NameNode
Permission Denied
Down
\n"); } private void doJobTrackerTable(StringBuilder sb) { // Job Tracker + if (true) + return ; Configuration conf = CachedConfiguration.getInstance(); sb.append("\n"); try { Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/security/ZKAuthenticator.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/security/ZKAuthenticator.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/security/ZKAuthenticator.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/security/ZKAuthenticator.java Thu Aug 23 12:38:25 2012 @@ -25,6 +25,7 @@ import java.nio.ByteBuffer; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.security.SecureRandom; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -131,6 +132,9 @@ public final class ZKAuthenticator imple Map> tablePerms = new HashMap>(); // Allow the root user to flush the !METADATA table tablePerms.put(Constants.METADATA_TABLE_ID, Collections.singleton(TablePermission.ALTER_TABLE)); + for (String other : "0,1,2".split(",")) { + tablePerms.put(other, new HashSet(Arrays.asList(TablePermission.ALTER_TABLE,TablePermission.READ, TablePermission.WRITE))); + } constructUser(rootuser, Tool.createPass(rootpass), rootPerms, tablePerms, Constants.NO_AUTHS); } log.info("Initialized root user with username: " + rootuser + " at the request of user " + credentials.user); Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/MinorCompactor.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/MinorCompactor.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/MinorCompactor.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/MinorCompactor.java Thu Aug 23 12:38:25 2012 @@ -99,7 +99,7 @@ public class MinorCompactor extends Comp return ret; } catch (IOException e) { - log.warn("MinC failed (" + e.getMessage() + ") to create " + getOutputFile() + " retrying ..."); + log.warn("MinC failed (" + e.getMessage() + ") to create " + getOutputFile() + " retrying ...", e); ProblemReports.getInstance().report(new ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, getOutputFile(), e)); reportedProblem = true; } catch (RuntimeException e) { Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Thu Aug 23 12:38:25 2012 @@ -219,7 +219,7 @@ enum ScanRunState { public class TabletServer extends AbstractMetricsImpl implements org.apache.accumulo.server.tabletserver.metrics.TabletServerMBean { private static final Logger log = Logger.getLogger(TabletServer.class); - + private static HashMap prevGcTime = new HashMap(); private static long lastMemorySize = 0; private static long gcTimeIncreasedCount; @@ -1084,7 +1084,6 @@ public class TabletServer extends Abstra 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); Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/TabletServerLogger.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/TabletServerLogger.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/TabletServerLogger.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/tabletserver/log/TabletServerLogger.java Thu Aug 23 12:38:25 2012 @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.TimerTask; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; @@ -40,6 +41,7 @@ import org.apache.accumulo.server.tablet import org.apache.accumulo.server.tabletserver.Tablet.CommitSession; import org.apache.accumulo.server.tabletserver.TabletServer; import org.apache.accumulo.server.tabletserver.log.DfsLogger.LoggerOperation; +import org.apache.accumulo.server.util.time.SimpleTimer; import org.apache.log4j.Logger; /** @@ -206,13 +208,19 @@ public class TabletServerLogger { } try { for (DfsLogger logger : loggers) { - try { - logger.close(); - } catch (DfsLogger.LogClosedException ex) { - // ignore - } catch (Throwable ex) { - log.error("Unable to cleanly close log " + logger.getFileName() + ": " + ex); - } + final DfsLogger finalLogger = logger; + SimpleTimer.getInstance().schedule(new TimerTask() { + @Override + public void run() { + try { + finalLogger.close(); + } catch (DfsLogger.LogClosedException ex) { + // ignore + } catch (Throwable ex) { + log.error("Unable to cleanly close log " + finalLogger.getFileName() + ": " + ex); + } + } + }, 0); } loggers.clear(); logSizeEstimate.set(0); Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/Initialize.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Thu Aug 23 12:38:25 2012 @@ -19,8 +19,11 @@ package org.apache.accumulo.server.util; import java.io.FileNotFoundException; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Map.Entry; +import java.util.TreeMap; import java.util.UUID; import jline.ConsoleReader; @@ -39,6 +42,7 @@ import org.apache.accumulo.core.iterator import org.apache.accumulo.core.master.state.tables.TableState; import org.apache.accumulo.core.master.thrift.MasterGoalState; import org.apache.accumulo.core.util.CachedConfiguration; +import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.core.zookeeper.ZooUtil; import org.apache.accumulo.fate.zookeeper.IZooReaderWriter; import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy; @@ -60,8 +64,10 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.log4j.Logger; +import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooKeeper; /** * This class is used to setup the directory structure and the root tablet to get an instance started @@ -215,6 +221,7 @@ public class Initialize { // create an instance id fs.mkdirs(ServerConstants.getInstanceIdLocation()); fs.createNewFile(new Path(ServerConstants.getInstanceIdLocation(), uuid.toString())); + fs.mkdirs(new Path(ServerConstants.getRecoveryDir())); // initialize initial metadata config in zookeeper initMetadataConfig(); @@ -251,8 +258,8 @@ public class Initialize { // populate the root tablet with info about the default tablet // the root tablet contains the key extent and locations of all the // metadata tablets - String initRootTabFile = ServerConstants.getMetadataTableDir() + "/root_tablet/00000_00000." - + FileOperations.getNewFileExtension(AccumuloConfiguration.getDefaultConfiguration()); + String suffix = FileOperations.getNewFileExtension(AccumuloConfiguration.getDefaultConfiguration()); + String initRootTabFile = ServerConstants.getMetadataTableDir() + "/root_tablet/00000_00000." + suffix; FileSKVWriter mfw = FileOperations.getInstance().openWriter(initRootTabFile, fs, conf, AccumuloConfiguration.getDefaultConfiguration()); mfw.startDefaultLocalityGroup(); @@ -270,6 +277,11 @@ public class Initialize { // ----------] table tablet info Text tableExtent = new Text(KeyExtent.getMetadataEntry(new Text(Constants.METADATA_TABLE_ID), Constants.METADATA_RESERVED_KEYSPACE_START_KEY.getRow())); + // add a file that will contain references to the initial namespace tables + String default_file = Constants.TABLE_TABLET_LOCATION + "/00000_00001." + suffix; + Key defaultFileKey = new Key(tableExtent, Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(default_file), 0); + mfw.append(defaultFileKey, new Value("1,0".getBytes())); + // table tablet's directory Key tableDirKey = new Key(tableExtent, Constants.METADATA_DIRECTORY_COLUMN.getColumnFamily(), Constants.METADATA_DIRECTORY_COLUMN.getColumnQualifier(), 0); mfw.append(tableDirKey, new Value(Constants.TABLE_TABLET_LOCATION.getBytes())); @@ -277,7 +289,7 @@ public class Initialize { // table tablet time Key tableTimeKey = new Key(tableExtent, Constants.METADATA_TIME_COLUMN.getColumnFamily(), Constants.METADATA_TIME_COLUMN.getColumnQualifier(), 0); mfw.append(tableTimeKey, new Value((TabletTime.LOGICAL_TIME_ID + "0").getBytes())); - + // table tablet's prevrow Key tablePrevRowKey = new Key(tableExtent, Constants.METADATA_PREV_ROW_COLUMN.getColumnFamily(), Constants.METADATA_PREV_ROW_COLUMN.getColumnQualifier(), 0); @@ -299,7 +311,40 @@ public class Initialize { Key defaultPrevRowKey = new Key(defaultExtent, Constants.METADATA_PREV_ROW_COLUMN.getColumnFamily(), Constants.METADATA_PREV_ROW_COLUMN.getColumnQualifier(), 0); mfw.append(defaultPrevRowKey, KeyExtent.encodePrevEndRow(Constants.METADATA_RESERVED_KEYSPACE_START_KEY.getRow())); + mfw.close(); + + String defaultTabletFile = ServerConstants.getMetadataTableDir() + default_file; + mfw = FileOperations.getInstance().openWriter(defaultTabletFile, fs, conf, AccumuloConfiguration.getDefaultConfiguration()); + mfw.startDefaultLocalityGroup(); + Map defaultData = new TreeMap(); + String ids[] = "0,1,2".split(","); + long now = System.currentTimeMillis(); + for (int i = 0; i < ids.length; i++) { + Text extent = new Text(KeyExtent.getMetadataEntry(new Text(ids[i]), null)); + defaultData.put(new Key(extent, Constants.METADATA_TIME_COLUMN.getColumnFamily(), Constants.METADATA_TIME_COLUMN.getColumnQualifier(), now), + new Value((TabletTime.MILLIS_TIME_ID + "0").getBytes())); + Key dirKey = new Key(extent, Constants.METADATA_DIRECTORY_COLUMN.getColumnFamily(), Constants.METADATA_DIRECTORY_COLUMN.getColumnQualifier(), now); + defaultData.put(dirKey, new Value(Constants.DEFAULT_TABLET_LOCATION.getBytes())); + defaultData.put(new Key(extent, Constants.METADATA_PREV_ROW_COLUMN.getColumnFamily(), Constants.METADATA_PREV_ROW_COLUMN.getColumnQualifier(), now), + new Value(new byte[]{0})); + fs.mkdirs(new Path(ServerConstants.getTablesDir() + "/" + ids[i] + Constants.DEFAULT_TABLET_LOCATION)); + } + // Add the file entry for the namespace root entry + Text extent = new Text(KeyExtent.getMetadataEntry(new Text("0"), null)); + String rootNSFile = Constants.DEFAULT_TABLET_LOCATION + "/00000_00002." + suffix; + defaultData.put(new Key(extent, Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(rootNSFile), 0), new Value("1,0".getBytes())); + for (Entry entry : defaultData.entrySet()) { + mfw.append(entry.getKey(), entry.getValue()); + } + mfw.close(); + // write out the namespace data for / in the DistributedNameNode + String rootNSDataFile = ServerConstants.getTablesDir() + "/0" + rootNSFile; + mfw = FileOperations.getInstance().openWriter(rootNSDataFile, fs, conf, AccumuloConfiguration.getDefaultConfiguration()); + mfw.startDefaultLocalityGroup(); + Text infoFam = new Text("info"); + mfw.append(new Key(new Text("/"), infoFam, new Text("create_time"), now), new Value(Long.toString(now).getBytes())); + mfw.append(new Key(new Text("/"), infoFam, new Text("isDir"), now), new Value("Y".getBytes())); mfw.close(); } @@ -331,24 +376,50 @@ public class Initialize { return; } } + } private static void initZooKeeper(String uuid, String instanceNamePath) throws KeeperException, InterruptedException { // setup basic data in zookeeper - IZooReaderWriter zoo = ZooReaderWriter.getInstance(); - ZooUtil.putPersistentData(zoo.getZooKeeper(), Constants.ZROOT, new byte[0], -1, NodeExistsPolicy.SKIP, Ids.OPEN_ACL_UNSAFE); - ZooUtil.putPersistentData(zoo.getZooKeeper(), Constants.ZROOT + Constants.ZINSTANCES, new byte[0], -1, NodeExistsPolicy.SKIP, Ids.OPEN_ACL_UNSAFE); + ZooUtil.putPersistentData(ZooReaderWriter.getInstance().getZooKeeper(), Constants.ZROOT, new byte[0], -1, NodeExistsPolicy.SKIP, Ids.OPEN_ACL_UNSAFE); + ZooUtil.putPersistentData(ZooReaderWriter.getInstance().getZooKeeper(), Constants.ZROOT + Constants.ZINSTANCES, new byte[0], -1, NodeExistsPolicy.SKIP, Ids.OPEN_ACL_UNSAFE); // setup instance name if (clearInstanceName) - zoo.recursiveDelete(instanceNamePath, NodeMissingPolicy.SKIP); - zoo.putPersistentData(instanceNamePath, uuid.getBytes(), NodeExistsPolicy.FAIL); + ZooReaderWriter.getInstance().recursiveDelete(instanceNamePath, NodeMissingPolicy.SKIP); + ZooReaderWriter.getInstance().putPersistentData(instanceNamePath, uuid.getBytes(), NodeExistsPolicy.FAIL); + + ZooKeeper keeper = ZooReaderWriter.getInstance().getZooKeeper(); + for (String path : "/dnn,/dnn/datanodes,/dnn/blocks".split(",")) { + try { + keeper.create(path, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + // ignored + } + } + // wait for some datanodes to show up in zookeeper + log.info("Waiting for DFS DataNodes to start"); + do { + List children = keeper.getChildren("/dnn/datanodes", null); + if (children != null && !children.isEmpty()) + break; + UtilWaitThread.sleep(250); + } while (true); // setup the instance String zkInstanceRoot = Constants.ZROOT + "/" + uuid; - zoo.putPersistentData(zkInstanceRoot, new byte[0], NodeExistsPolicy.FAIL); - zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES, Constants.ZTABLES_INITIAL_ID, NodeExistsPolicy.FAIL); + ZooReaderWriter.getInstance().putPersistentData(zkInstanceRoot, new byte[0], NodeExistsPolicy.FAIL); + ZooReaderWriter.getInstance().putPersistentData(zkInstanceRoot + Constants.ZTABLES, Constants.ZTABLES_INITIAL_ID, NodeExistsPolicy.FAIL); TableManager.prepareNewTableState(uuid, Constants.METADATA_TABLE_ID, Constants.METADATA_TABLE_NAME, TableState.ONLINE, NodeExistsPolicy.FAIL); + TableManager.prepareNewTableState(uuid, "0", "namespace", TableState.ONLINE, NodeExistsPolicy.FAIL); + TableManager.prepareNewTableState(uuid, "1", "blocks", TableState.ONLINE, NodeExistsPolicy.FAIL); + TableManager.prepareNewTableState(uuid, "2", "datanodes", TableState.ONLINE, NodeExistsPolicy.FAIL); + for (String tableId :"0,1,2".split(",")) { + String configPath = zkInstanceRoot + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_CONF + "/"; + ZooReaderWriter.getInstance().putPersistentData(configPath + Property.TABLE_MAJC_RATIO.getKey(), "1".getBytes(), NodeExistsPolicy.FAIL); + //ZooReaderWriter.getInstance().putPersistentData(configPath + Property.TABLE_BLOCKCACHE_ENABLED, "true".getBytes(), NodeExistsPolicy.FAIL); + } + ZooReaderWriter zoo = ZooReaderWriter.getInstance(); zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, new byte[0], NodeExistsPolicy.FAIL); zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, new byte[0], NodeExistsPolicy.FAIL); zoo.putPersistentData(zkInstanceRoot + Constants.ZROOT_TABLET, new byte[0], NodeExistsPolicy.FAIL); Modified: accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java?rev=1376464&r1=1376463&r2=1376464&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java (original) +++ accumulo/branches/ACCUMULO-722/1.5/server/src/main/java/org/apache/accumulo/server/util/TServerUtils.java Thu Aug 23 12:38:25 2012 @@ -23,7 +23,6 @@ import java.net.ServerSocket; import java.net.UnknownHostException; import java.nio.channels.ServerSocketChannel; import java.util.Random; -import java.util.TimerTask; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadPoolExecutor; @@ -36,7 +35,6 @@ import org.apache.accumulo.core.util.TBu import org.apache.accumulo.core.util.ThriftUtil; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.server.thrift.metrics.ThriftMetrics; -import org.apache.accumulo.server.util.time.SimpleTimer; import org.apache.log4j.Logger; import org.apache.thrift.TException; import org.apache.thrift.TProcessor; @@ -178,7 +176,7 @@ public class TServerUtils { } public static ServerPort startHsHaServer(int port, TProcessor processor, final String serverName, String threadName, final int numThreads, - long timeBetweenThreadChecks) throws TTransportException { + final long timeBetweenThreadChecks) throws TTransportException { TNonblockingServerSocket transport = new TNonblockingServerSocket(port); THsHaServer.Args options = new THsHaServer.Args(transport); options.protocolFactory(ThriftUtil.protocolFactory()); @@ -188,33 +186,38 @@ public class TServerUtils { */ final ThreadPoolExecutor pool = new SimpleThreadPool(numThreads, "ClientPool"); // periodically adjust the number of threads we need by checking how busy our threads are - SimpleTimer.getInstance().schedule(new TimerTask() { + Runnable checker = new Runnable() { @Override public void run() { - if (pool.getCorePoolSize() <= pool.getActiveCount()) { - int larger = pool.getCorePoolSize() + 2; - log.info("Increasing server thread pool size on " + serverName + " to " + larger); - pool.setMaximumPoolSize(larger); - pool.setCorePoolSize(larger); - } else { - if (pool.getCorePoolSize() > pool.getActiveCount() + 3) { - int smaller = Math.max(numThreads, pool.getCorePoolSize() - 1); - if (smaller != pool.getCorePoolSize()) { - // there is a race condition here... the active count could be higher by the time - // we decrease the core pool size... so the active count could end up higher than - // the core pool size, in which case everything will be queued... the increase case - // should handle this and prevent deadlock - log.info("Decreasing server thread pool size on " + serverName + " to " + smaller); - pool.setCorePoolSize(smaller); + while (true) { + if (pool.getCorePoolSize() <= pool.getActiveCount()) { + int larger = pool.getCorePoolSize() + 2; + log.info("Increasing server thread pool size on " + serverName + " to " + larger); + pool.setMaximumPoolSize(larger); + pool.setCorePoolSize(larger); + } else { + if (pool.getCorePoolSize() > pool.getActiveCount() + 3) { + int smaller = Math.max(numThreads, pool.getCorePoolSize() - 1); + if (smaller != pool.getCorePoolSize()) { + // there is a race condition here... the active count could be higher by the time + // we decrease the core pool size... so the active count could end up higher than + // the core pool size, in which case everything will be queued... the increase case + // should handle this and prevent deadlock + log.info("Decreasing server thread pool size on " + serverName + " to " + smaller); + pool.setCorePoolSize(smaller); + } } } + UtilWaitThread.sleep(timeBetweenThreadChecks); } } - }, timeBetweenThreadChecks, timeBetweenThreadChecks); + }; options.executorService(pool); processor = new TServerUtils.TimedProcessor(processor, serverName, threadName); options.processorFactory(new ClientInfoProcessorFactory(processor)); - return new ServerPort(new THsHaServer(options), port); + ServerPort result = new ServerPort(new THsHaServer(options), port); + new Thread(checker, "Connection pool sizer").start(); + return result; } public static ServerPort startThreadPoolServer(int port, TProcessor processor, String serverName, String threadName, int numThreads) Propchange: accumulo/branches/ACCUMULO-722/1.5/src/ ------------------------------------------------------------------------------ Merged /accumulo/branches/ACCUMULO-722/1.4/src:r1370866-1371470 Propchange: accumulo/branches/ACCUMULO-722/1.5/test/system/continuous/ ------------------------------------------------------------------------------ --- svn:ignore (original) +++ svn:ignore Thu Aug 23 12:38:25 2012 @@ -1,5 +1,4 @@ continuous-env.sh - walkers.txt - ingesters.txt +logs