Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 07291200BB8 for ; Sat, 8 Oct 2016 01:16:47 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 05EFC160AE8; Fri, 7 Oct 2016 23:16:47 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 8C3CB160AF1 for ; Sat, 8 Oct 2016 01:16:45 +0200 (CEST) Received: (qmail 39546 invoked by uid 500); 7 Oct 2016 23:16:44 -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 39526 invoked by uid 99); 7 Oct 2016 23:16:44 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 07 Oct 2016 23:16:44 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 891F4E09AC; Fri, 7 Oct 2016 23:16:44 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ctubbsii@apache.org To: commits@accumulo.apache.org Date: Fri, 07 Oct 2016 23:16:45 -0000 Message-Id: <7d36575958ed48c1a9529fe0f43b7b75@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/6] accumulo git commit: ACCUMULO-4481 Remove unused logger.* properties archived-at: Fri, 07 Oct 2016 23:16:47 -0000 ACCUMULO-4481 Remove unused logger.* properties Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/5e4a8d90 Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/5e4a8d90 Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/5e4a8d90 Branch: refs/heads/master Commit: 5e4a8d907dc8c5a9bc6be41cc7980346d51a4b33 Parents: 272ac28 Author: Christopher Tubbs Authored: Thu Sep 29 17:51:59 2016 -0400 Committer: Christopher Tubbs Committed: Thu Sep 29 17:51:59 2016 -0400 ---------------------------------------------------------------------- assemble/conf/examples/crypto/accumulo-site.xml | 6 - .../examples/vfs-classloader/accumulo-site.xml | 9 - assemble/conf/templates/accumulo-site.xml | 9 - .../org/apache/accumulo/core/conf/Property.java | 15 +- .../test/resources/crypto-off-accumulo-site.xml | 6 - .../test/resources/crypto-on-accumulo-site.xml | 6 - ...rypto-on-no-key-encryption-accumulo-site.xml | 6 - .../resources/disabled/conf/accumulo-site.xml | 9 - .../main/asciidoc/chapters/troubleshooting.txt | 2 +- .../impl/MiniAccumuloConfigImpl.java | 1 - .../accumulo/tserver/log/LocalWALRecovery.java | 187 ------------------- .../tserver/log/LocalWALRecoveryTest.java | 100 ---------- test/src/main/resources/conf/accumulo-site.xml | 7 - 13 files changed, 5 insertions(+), 358 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/assemble/conf/examples/crypto/accumulo-site.xml ---------------------------------------------------------------------- diff --git a/assemble/conf/examples/crypto/accumulo-site.xml b/assemble/conf/examples/crypto/accumulo-site.xml index 82052d5..acc946f 100644 --- a/assemble/conf/examples/crypto/accumulo-site.xml +++ b/assemble/conf/examples/crypto/accumulo-site.xml @@ -35,12 +35,6 @@ - logger.dir.walog - walogs - The directory used to store write-ahead logs on the local filesystem. It is possible to specify a comma-separated list of directories. - - - instance.secret DEFAULT A secret unique to a given instance that all servers must know in order to communicate with one another. http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/assemble/conf/examples/vfs-classloader/accumulo-site.xml ---------------------------------------------------------------------- diff --git a/assemble/conf/examples/vfs-classloader/accumulo-site.xml b/assemble/conf/examples/vfs-classloader/accumulo-site.xml index fb66d27..7c41e4d 100644 --- a/assemble/conf/examples/vfs-classloader/accumulo-site.xml +++ b/assemble/conf/examples/vfs-classloader/accumulo-site.xml @@ -28,15 +28,6 @@ - logger.dir.walog - walogs - The property only needs to be set if upgrading from 1.4 which used to store write-ahead logs on the local - filesystem. In 1.5 write-ahead logs are stored in DFS. When 1.5 is started for the first time it will copy any 1.4 - write ahead logs into DFS. It is possible to specify a comma-separated list of directories. - - - - instance.secret DEFAULT A secret unique to a given instance that all servers must know in order to communicate with one another. http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/assemble/conf/templates/accumulo-site.xml ---------------------------------------------------------------------- diff --git a/assemble/conf/templates/accumulo-site.xml b/assemble/conf/templates/accumulo-site.xml index eaaf2b6..ff490b1 100644 --- a/assemble/conf/templates/accumulo-site.xml +++ b/assemble/conf/templates/accumulo-site.xml @@ -34,15 +34,6 @@ - logger.dir.walog - walogs - The property only needs to be set if upgrading from 1.4 which used to store write-ahead logs on the local - filesystem. In 1.5 write-ahead logs are stored in DFS. When 1.5 is started for the first time it will copy any 1.4 - write ahead logs into DFS. It is possible to specify a comma-separated list of directories. - - - - instance.secret DEFAULT A secret unique to a given instance that all servers must know in order to communicate with one another. http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/core/src/main/java/org/apache/accumulo/core/conf/Property.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java index ede1c6f..aa0a4b1 100644 --- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java +++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java @@ -361,13 +361,6 @@ public enum Property { TSERV_SLOW_FLUSH_MILLIS("tserver.slow.flush.time", "100ms", PropertyType.TIMEDURATION, "If a flush to the write-ahead log takes longer than this period of time, debugging information will written, and may result in a log rollover."), - // properties that are specific to logger server behavior - LOGGER_PREFIX("logger.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the write-ahead logger servers"), - LOGGER_DIR("logger.dir.walog", "walogs", PropertyType.PATH, "This property is only needed if Accumulo was upgraded from a 1.4 or earlier version. " - + "In the upgrade to 1.5 this property is used to copy any earlier write ahead logs into DFS. " - + "In 1.6+, this property is used by the LocalWALRecovery utility in the event that something went wrong with that earlier upgrade. " - + "It is possible to specify a comma-separated list of directories."), - // accumulo garbage collector properties GC_PREFIX("gc.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the accumulo garbage collector."), GC_CYCLE_START("gc.cycle.start", "30s", PropertyType.TIMEDURATION, "Time to wait before attempting to garbage collect any old files."), @@ -856,10 +849,10 @@ public enum Property { */ public static boolean isValidZooPropertyKey(String key) { // white list prefixes - return key.startsWith(Property.TABLE_PREFIX.getKey()) || key.startsWith(Property.TSERV_PREFIX.getKey()) || key.startsWith(Property.LOGGER_PREFIX.getKey()) - || key.startsWith(Property.MASTER_PREFIX.getKey()) || key.startsWith(Property.GC_PREFIX.getKey()) - || key.startsWith(Property.MONITOR_PREFIX.getKey() + "banner.") || key.startsWith(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey()) - || key.startsWith(Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey()) || key.startsWith(REPLICATION_PREFIX.getKey()); + return key.startsWith(Property.TABLE_PREFIX.getKey()) || key.startsWith(Property.TSERV_PREFIX.getKey()) || key.startsWith(Property.MASTER_PREFIX.getKey()) + || key.startsWith(Property.GC_PREFIX.getKey()) || key.startsWith(Property.MONITOR_PREFIX.getKey() + "banner.") + || key.startsWith(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey()) || key.startsWith(Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey()) + || key.startsWith(REPLICATION_PREFIX.getKey()); } /** http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/core/src/test/resources/crypto-off-accumulo-site.xml ---------------------------------------------------------------------- diff --git a/core/src/test/resources/crypto-off-accumulo-site.xml b/core/src/test/resources/crypto-off-accumulo-site.xml index 667e9a3..ca4bef9 100644 --- a/core/src/test/resources/crypto-off-accumulo-site.xml +++ b/core/src/test/resources/crypto-off-accumulo-site.xml @@ -35,12 +35,6 @@ - logger.dir.walog - walogs - The directory used to store write-ahead logs on the local filesystem. It is possible to specify a comma-separated list of directories. - - - instance.secret DEFAULT A secret unique to a given instance that all servers must know in order to communicate with one another. http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/core/src/test/resources/crypto-on-accumulo-site.xml ---------------------------------------------------------------------- diff --git a/core/src/test/resources/crypto-on-accumulo-site.xml b/core/src/test/resources/crypto-on-accumulo-site.xml index 3da7437..61b64a9 100644 --- a/core/src/test/resources/crypto-on-accumulo-site.xml +++ b/core/src/test/resources/crypto-on-accumulo-site.xml @@ -35,12 +35,6 @@ - logger.dir.walog - walogs - The directory used to store write-ahead logs on the local filesystem. It is possible to specify a comma-separated list of directories. - - - instance.secret DEFAULT A secret unique to a given instance that all servers must know in order to communicate with one another. http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml ---------------------------------------------------------------------- diff --git a/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml b/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml index edbcfeb..128a54f 100644 --- a/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml +++ b/core/src/test/resources/crypto-on-no-key-encryption-accumulo-site.xml @@ -35,12 +35,6 @@ - logger.dir.walog - walogs - The directory used to store write-ahead logs on the local filesystem. It is possible to specify a comma-separated list of directories. - - - instance.secret DEFAULT A secret unique to a given instance that all servers must know in order to communicate with one another. http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/core/src/test/resources/disabled/conf/accumulo-site.xml ---------------------------------------------------------------------- diff --git a/core/src/test/resources/disabled/conf/accumulo-site.xml b/core/src/test/resources/disabled/conf/accumulo-site.xml index 78e0c7a..6c9855d 100644 --- a/core/src/test/resources/disabled/conf/accumulo-site.xml +++ b/core/src/test/resources/disabled/conf/accumulo-site.xml @@ -30,15 +30,6 @@ - logger.dir.walog - walogs - The directory used to store write-ahead logs on the local - filesystem. It is possible to specify a comma-separated list of - directories. - - - - instance.secret DEFAULT A secret unique to a given instance that all servers must http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/docs/src/main/asciidoc/chapters/troubleshooting.txt ---------------------------------------------------------------------- diff --git a/docs/src/main/asciidoc/chapters/troubleshooting.txt b/docs/src/main/asciidoc/chapters/troubleshooting.txt index 667303f..d993767 100644 --- a/docs/src/main/asciidoc/chapters/troubleshooting.txt +++ b/docs/src/main/asciidoc/chapters/troubleshooting.txt @@ -780,7 +780,7 @@ omission of new data. *Q*: I upgraded from 1.4 to 1.5 to 1.6 but still have some WAL files on local disk. Do I have any way to recover them? -*A*: Yes, you can recover them by running the LocalWALRecovery utility on each node that needs recovery performed. The utility +*A*: Yes, you can recover them by running the LocalWALRecovery utility (not available in 1.8 and later) on each node that needs recovery performed. The utility will default to using the directory specified by +logger.dir.walog+ in your configuration, or can be overriden by using the +--local-wal-directories+ option on the tool. It can be invoked as follows: http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java ---------------------------------------------------------------------- diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java index 8e35705..20763bb 100644 --- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java +++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java @@ -133,7 +133,6 @@ public class MiniAccumuloConfigImpl { // TODO ACCUMULO-XXXX replace usage of instance.dfs.{dir,uri} with instance.volumes setInstanceLocation(); mergeProp(Property.INSTANCE_SECRET.getKey(), DEFAULT_INSTANCE_SECRET); - mergeProp(Property.LOGGER_DIR.getKey(), walogDir.getAbsolutePath()); mergeProp(Property.TRACE_TOKEN_PROPERTY_PREFIX.getKey() + "password", getRootPassword()); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java deleted file mode 100644 index 2667b53..0000000 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.accumulo.tserver.log; - -import java.io.EOFException; -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; - -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.conf.SiteConfiguration; -import org.apache.accumulo.server.ServerConstants; -import org.apache.accumulo.server.fs.VolumeManagerImpl; -import org.apache.accumulo.server.security.SecurityUtil; -import org.apache.accumulo.tserver.logger.LogFileKey; -import org.apache.accumulo.tserver.logger.LogFileValue; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.SequenceFile.Reader; -import org.apache.hadoop.io.WritableName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import com.beust.jcommander.ParameterException; -import com.google.common.annotations.VisibleForTesting; - -/** - * This class will attempt to rewrite any local WALs to HDFS. - */ -@SuppressWarnings("deprecation") -public class LocalWALRecovery implements Runnable { - private static final Logger log = LoggerFactory.getLogger(LocalWALRecovery.class); - - static { - WritableName.addName(LogFileKey.class, "org.apache.accumulo.server.logger.LogFileKey"); - WritableName.addName(LogFileValue.class, "org.apache.accumulo.server.logger.LogFileValue"); - } - - public static void main(String[] args) throws IOException { - AccumuloConfiguration configuration = SiteConfiguration.getInstance(SiteConfiguration.getDefaultConfiguration()); - - LocalWALRecovery main = new LocalWALRecovery(configuration); - main.parseArgs(args); - main.run(); - } - - public final class Options { - @Parameter(names = "--delete-local", description = "Specify whether to delete the local WAL files after they have been re-written in HDFS.") - public boolean deleteLocal = false; - - @Parameter(names = "--local-wal-directories", - description = "Comma separated list of local directories containing WALs, default is set according to the logger.dir.walog property.") - public List directories = getDefaultDirectories(); - - @Parameter(names = "--dfs-wal-directory", - description = "The directory that WALs will be copied into. Will default to the first configured base dir + '/wal'") - public String destination = null; - - private List getDefaultDirectories() { - String property = configuration.get(Property.LOGGER_DIR); - return Arrays.asList(property.split(",")); - } - } - - private final AccumuloConfiguration configuration; - private final Options options; - - /** - * Create a WAL recovery tool for the given instance. - */ - public LocalWALRecovery(AccumuloConfiguration configuration) { - this.configuration = configuration; - this.options = new Options(); - } - - @VisibleForTesting - public void parseArgs(String... args) { - JCommander jcommander = new JCommander(); - jcommander.addObject(options); - - try { - jcommander.parse(args); - } catch (ParameterException e) { - jcommander.usage(); - } - } - - @Override - public void run() { - SecurityUtil.serverLogin(SiteConfiguration.getInstance()); - - try { - recoverLocalWriteAheadLogs(VolumeManagerImpl.get().getDefaultVolume().getFileSystem()); - } catch (IOException e) { - log.error("Error while recovering WAL files.", e); - } - } - - public void recoverLocalWriteAheadLogs(FileSystem fs) throws IOException { - for (String directory : options.directories) { - File localDirectory = new File(directory); - if (!localDirectory.isAbsolute()) { - localDirectory = new File(System.getenv("ACCUMULO_HOME"), directory); - } - - if (!localDirectory.isDirectory()) { - log.warn("Local walog dir " + localDirectory.getAbsolutePath() + " does not exist or is not a directory."); - continue; - } - - if (options.destination == null) { - // Defer loading the default value until now because it might require talking to zookeeper. - options.destination = ServerConstants.getWalDirs()[0]; - } - log.info("Copying WALs to " + options.destination); - - File[] files = localDirectory.listFiles(); - if (files != null) { - for (File file : files) { - String name = file.getName(); - try { - UUID.fromString(name); - } catch (IllegalArgumentException ex) { - log.info("Ignoring non-log file " + file.getAbsolutePath()); - continue; - } - - LogFileKey key = new LogFileKey(); - LogFileValue value = new LogFileValue(); - - log.info("Openning local log " + file.getAbsolutePath()); - - Path localWal = new Path(file.toURI()); - FileSystem localFs = FileSystem.getLocal(fs.getConf()); - - Reader reader = new SequenceFile.Reader(localFs, localWal, localFs.getConf()); - // Reader reader = new SequenceFile.Reader(localFs.getConf(), SequenceFile.Reader.file(localWal)); - Path tmp = new Path(options.destination + "/" + name + ".copy"); - FSDataOutputStream writer = fs.create(tmp); - while (reader.next(key, value)) { - try { - key.write(writer); - value.write(writer); - } catch (EOFException ex) { - break; - } - } - writer.close(); - reader.close(); - fs.rename(tmp, new Path(tmp.getParent(), name)); - - if (options.deleteLocal) { - if (file.delete()) { - log.info("Copied and deleted: " + name); - } else { - log.info("Failed to delete: " + name + " (but it is safe for you to delete it manually)."); - } - } else { - log.info("Safe to delete: " + name); - } - } - } - } - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java deleted file mode 100644 index 8261a17..0000000 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.accumulo.tserver.log; - -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; -import static org.junit.Assert.assertEquals; - -import java.io.DataInputStream; -import java.io.EOFException; -import java.io.File; -import java.io.IOException; - -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.server.fs.VolumeManager; -import org.apache.accumulo.server.fs.VolumeManagerImpl; -import org.apache.accumulo.tserver.log.DfsLogger.DFSLoggerInputStreams; -import org.apache.accumulo.tserver.logger.LogFileKey; -import org.apache.accumulo.tserver.logger.LogFileValue; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class LocalWALRecoveryTest { - - @Rule - public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target")); - - LocalWALRecovery recovery; - - File walTarget; - AccumuloConfiguration configuration; - - @Before - public void setUp() throws Exception { - File source = new File("src/test/resources", "walog-from-14"); - - configuration = createMock(AccumuloConfiguration.class); - expect(configuration.get(Property.LOGGER_DIR)).andReturn(source.getAbsolutePath()).anyTimes(); - replay(configuration); - - walTarget = folder.newFolder("wal"); - - recovery = new LocalWALRecovery(configuration); - recovery.parseArgs("--dfs-wal-directory", walTarget.getAbsolutePath()); - } - - @Test - public void testRecoverLocalWriteAheadLogs() throws IOException { - Path targetPath = new Path(walTarget.toURI()); - FileSystem fs = FileSystem.get(targetPath.toUri(), new Configuration()); - recovery.recoverLocalWriteAheadLogs(fs); - - FileStatus[] recovered = fs.listStatus(targetPath); - assertEquals("Wrong number of WAL files recovered.", 1, recovered.length); - - final Path path = recovered[0].getPath(); - final VolumeManager volumeManager = VolumeManagerImpl.getLocal(folder.getRoot().getAbsolutePath()); - - final DFSLoggerInputStreams streams = DfsLogger.readHeaderAndReturnStream(volumeManager, path, configuration); - final DataInputStream input = streams.getDecryptingInputStream(); - - final LogFileKey key = new LogFileKey(); - final LogFileValue value = new LogFileValue(); - int read = 0; - - while (true) { - try { - key.readFields(input); - value.readFields(input); - read++; - } catch (EOFException ex) { - break; - } - } - - assertEquals(104, read); - } -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/5e4a8d90/test/src/main/resources/conf/accumulo-site.xml ---------------------------------------------------------------------- diff --git a/test/src/main/resources/conf/accumulo-site.xml b/test/src/main/resources/conf/accumulo-site.xml index e1f0e70..23b47bf 100644 --- a/test/src/main/resources/conf/accumulo-site.xml +++ b/test/src/main/resources/conf/accumulo-site.xml @@ -35,13 +35,6 @@ - logger.dir.walog - walogs - The directory used to store write-ahead logs on the local filesystem. It is possible to specify a comma-separated list of directories. - - - - instance.secret DEFAULT A secret unique to a given instance that all servers must know in order to communicate with one another.