From commits-return-6153-archive-asf-public=cust-asf.ponee.io@zookeeper.apache.org Tue Feb 20 20:28:47 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 73856180654 for ; Tue, 20 Feb 2018 20:28:46 +0100 (CET) Received: (qmail 30248 invoked by uid 500); 20 Feb 2018 19:28:45 -0000 Mailing-List: contact commits-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list commits@zookeeper.apache.org Received: (qmail 30237 invoked by uid 99); 20 Feb 2018 19:28:45 -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; Tue, 20 Feb 2018 19:28:45 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 65A3EDFC25; Tue, 20 Feb 2018 19:28:45 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: afine@apache.org To: commits@zookeeper.apache.org Message-Id: <9af0ec36aecd44808f8130e0e10b8110@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: zookeeper git commit: ZOOKEEPER-2967: Add check to validate dataDir and dataLogDir parameters at startup Date: Tue, 20 Feb 2018 19:28:45 +0000 (UTC) Repository: zookeeper Updated Branches: refs/heads/branch-3.4 c9da2e287 -> 997fa5e76 ZOOKEEPER-2967: Add check to validate dataDir and dataLogDir parameters at startup ZOOKEEPER-2967: Add check to validate dataDir and dataLogDir parameters at startup This PR adds a check to protect ZK against configuring dataDir and dataLogDir opposingly. When FileTxnSnapLog is created, it checks if transaction log directory contains snapshot files or vice versa, snapshot directory contains transaction log files. If so, the check throws LogdirContentCheckException or SnapdirContentCheckException, respectively, which translates to DatadirException at ZK startup in QuorumPeerMain and ZooKeeperServerMain. If the two directories are the same, then no check is done. For testing, I've added 4 new unit tests which cover the following cases: transaction log and snapshot directories are different and they are used correctly (no Exception) transaction log and snapshot directories are the same (in this case no check is done) transaction log and snapshot directories are different and transaction log directory contains snapshot files (LogdirContentCheckException -> ZK quits) transaction log and snapshot directories are different and snapshot directory contains transaction log files (SnapdirContentCheckException -> ZK quits) Author: Mark Fenes Reviewers: Andor Molnár , Abraham Fine Closes #459 from mfenes/ZOOKEEPER-2967_3.4 Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/997fa5e7 Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/997fa5e7 Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/997fa5e7 Branch: refs/heads/branch-3.4 Commit: 997fa5e765f3a3288d4c010a9bb170dd59007031 Parents: c9da2e2 Author: Mark Fenes Authored: Tue Feb 20 11:28:37 2018 -0800 Committer: Abraham Fine Committed: Tue Feb 20 11:28:37 2018 -0800 ---------------------------------------------------------------------- .../zookeeper/server/persistence/FileSnap.java | 11 +- .../server/persistence/FileTxnLog.java | 26 ++-- .../server/persistence/FileTxnSnapLog.java | 58 +++++++- .../zookeeper/server/persistence/Util.java | 26 +++- .../server/persistence/FileTxnSnapLogTest.java | 148 +++++++++++++++++++ .../test/AtomicFileOutputStreamTest.java | 2 +- .../org/apache/zookeeper/test/ClientBase.java | 23 ++- .../org/apache/zookeeper/test/TestUtils.java | 60 ++++++++ 8 files changed, 329 insertions(+), 25 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java ---------------------------------------------------------------------- diff --git a/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java b/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java index cf464a1..1687277 100644 --- a/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java +++ b/src/java/main/org/apache/zookeeper/server/persistence/FileSnap.java @@ -56,6 +56,9 @@ public class FileSnap implements SnapShot { private static final Logger LOG = LoggerFactory.getLogger(FileSnap.class); public final static int SNAP_MAGIC = ByteBuffer.wrap("ZKSN".getBytes()).getInt(); + + public static final String SNAPSHOT_FILE_PREFIX = "snapshot"; + public FileSnap(File snapDir) { this.snapDir = snapDir; } @@ -104,7 +107,7 @@ public class FileSnap implements SnapShot { if (!foundValid) { throw new IOException("Not able to find valid snapshots in " + snapDir); } - dt.lastProcessedZxid = Util.getZxidFromName(snap.getName(), "snapshot"); + dt.lastProcessedZxid = Util.getZxidFromName(snap.getName(), SNAPSHOT_FILE_PREFIX); return dt.lastProcessedZxid; } @@ -152,7 +155,7 @@ public class FileSnap implements SnapShot { * @throws IOException */ private List findNValidSnapshots(int n) throws IOException { - List files = Util.sortDataDir(snapDir.listFiles(),"snapshot", false); + List files = Util.sortDataDir(snapDir.listFiles(), SNAPSHOT_FILE_PREFIX, false); int count = 0; List list = new ArrayList(); for (File f : files) { @@ -182,13 +185,13 @@ public class FileSnap implements SnapShot { * @throws IOException */ public List findNRecentSnapshots(int n) throws IOException { - List files = Util.sortDataDir(snapDir.listFiles(), "snapshot", false); + List files = Util.sortDataDir(snapDir.listFiles(), SNAPSHOT_FILE_PREFIX, false); int count = 0; List list = new ArrayList(); for (File f: files) { if (count == n) break; - if (Util.getZxidFromName(f.getName(), "snapshot") != -1) { + if (Util.getZxidFromName(f.getName(), SNAPSHOT_FILE_PREFIX) != -1) { count++; list.add(f); } http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java ---------------------------------------------------------------------- diff --git a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java index 0a70ea0..3694c98 100644 --- a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java +++ b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java @@ -99,6 +99,8 @@ public class FileTxnLog implements TxnLog { public final static int VERSION = 2; + public static final String LOG_FILE_PREFIX = "log"; + /** Maximum time we allow for elapsed fsync before WARNing */ private final static long fsyncWarningThresholdMS; @@ -207,13 +209,11 @@ public class FileTxnLog implements TxnLog { } if (logStream==null) { - if(LOG.isInfoEnabled()){ - LOG.info("Creating new log file: log." + - Long.toHexString(hdr.getZxid())); - } + if(LOG.isInfoEnabled()){ + LOG.info("Creating new log file: " + Util.makeLogName(hdr.getZxid())); + } - logFileWrite = new File(logDir, ("log." + - Long.toHexString(hdr.getZxid()))); + logFileWrite = new File(logDir, Util.makeLogName(hdr.getZxid())); fos = new FileOutputStream(logFileWrite); logStream=new BufferedOutputStream(fos); oa = BinaryOutputArchive.getArchive(logStream); @@ -290,12 +290,12 @@ public class FileTxnLog implements TxnLog { * @return */ public static File[] getLogFiles(File[] logDirList,long snapshotZxid) { - List files = Util.sortDataDir(logDirList, "log", true); + List files = Util.sortDataDir(logDirList, LOG_FILE_PREFIX, true); long logZxid = 0; // Find the log file that starts before or at the same time as the // zxid of the snapshot for (File f : files) { - long fzxid = Util.getZxidFromName(f.getName(), "log"); + long fzxid = Util.getZxidFromName(f.getName(), LOG_FILE_PREFIX); if (fzxid > snapshotZxid) { continue; } @@ -307,7 +307,7 @@ public class FileTxnLog implements TxnLog { } List v=new ArrayList(5); for (File f : files) { - long fzxid = Util.getZxidFromName(f.getName(), "log"); + long fzxid = Util.getZxidFromName(f.getName(), LOG_FILE_PREFIX); if (fzxid < logZxid) { continue; } @@ -324,7 +324,7 @@ public class FileTxnLog implements TxnLog { public long getLastLoggedZxid() { File[] files = getLogFiles(logDir.listFiles(), 0); long maxLog=files.length>0? - Util.getZxidFromName(files[files.length-1].getName(),"log"):-1; + Util.getZxidFromName(files[files.length-1].getName(),LOG_FILE_PREFIX):-1; // if a log file is more recent we must scan it to find // the highest zxid @@ -578,13 +578,13 @@ public class FileTxnLog implements TxnLog { */ void init() throws IOException { storedFiles = new ArrayList(); - List files = Util.sortDataDir(FileTxnLog.getLogFiles(logDir.listFiles(), 0), "log", false); + List files = Util.sortDataDir(FileTxnLog.getLogFiles(logDir.listFiles(), 0), LOG_FILE_PREFIX, false); for (File f: files) { - if (Util.getZxidFromName(f.getName(), "log") >= zxid) { + if (Util.getZxidFromName(f.getName(), LOG_FILE_PREFIX) >= zxid) { storedFiles.add(f); } // add the last logfile that is less than the zxid - else if (Util.getZxidFromName(f.getName(), "log") < zxid) { + else if (Util.getZxidFromName(f.getName(), LOG_FILE_PREFIX) < zxid) { storedFiles.add(f); break; } http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java ---------------------------------------------------------------------- diff --git a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java index 24614c0..b261a8e 100644 --- a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java +++ b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java @@ -19,6 +19,7 @@ package org.apache.zookeeper.server.persistence; import java.io.File; +import java.io.FilenameFilter; import java.io.IOException; import java.util.List; import java.util.Map; @@ -100,10 +101,41 @@ public class FileTxnSnapLog { throw new IOException("Cannot write to snap directory " + this.snapDir); } + // check content of transaction log and snapshot dirs if they are two different directories + // See ZOOKEEPER-2967 for more details + if(!this.dataDir.getPath().equals(this.snapDir.getPath())){ + checkLogDir(); + checkSnapDir(); + } + txnLog = new FileTxnLog(this.dataDir); snapLog = new FileSnap(this.snapDir); } - + + private void checkLogDir() throws LogDirContentCheckException { + File[] files = this.dataDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return Util.isSnapshotFileName(name); + } + }); + if (files != null && files.length > 0) { + throw new LogDirContentCheckException("Log directory has snapshot files. Check if dataLogDir and dataDir configuration is correct."); + } + } + + private void checkSnapDir() throws SnapDirContentCheckException { + File[] files = this.snapDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return Util.isLogFileName(name); + } + }); + if (files != null && files.length > 0) { + throw new SnapDirContentCheckException("Snapshot directory has log files. Check if dataLogDir and dataDir configuration is correct."); + } + } + /** * get the datadir used by this filetxn * snap log @@ -348,4 +380,28 @@ public class FileTxnSnapLog { txnLog.close(); snapLog.close(); } + + @SuppressWarnings("serial") + public static class DatadirException extends IOException { + public DatadirException(String msg) { + super(msg); + } + public DatadirException(String msg, Exception e) { + super(msg, e); + } + } + + @SuppressWarnings("serial") + public static class LogDirContentCheckException extends DatadirException { + public LogDirContentCheckException(String msg) { + super(msg); + } + } + + @SuppressWarnings("serial") + public static class SnapDirContentCheckException extends DatadirException { + public SnapDirContentCheckException(String msg) { + super(msg); + } + } } http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/main/org/apache/zookeeper/server/persistence/Util.java ---------------------------------------------------------------------- diff --git a/src/java/main/org/apache/zookeeper/server/persistence/Util.java b/src/java/main/org/apache/zookeeper/server/persistence/Util.java index 656c2e8..a774a24 100644 --- a/src/java/main/org/apache/zookeeper/server/persistence/Util.java +++ b/src/java/main/org/apache/zookeeper/server/persistence/Util.java @@ -83,7 +83,7 @@ public class Util { * @return file name */ public static String makeLogName(long zxid) { - return "log." + Long.toHexString(zxid); + return FileTxnLog.LOG_FILE_PREFIX + "." + Long.toHexString(zxid); } /** @@ -93,7 +93,7 @@ public class Util { * @return file name */ public static String makeSnapshotName(long zxid) { - return "snapshot." + Long.toHexString(zxid); + return FileSnap.SNAPSHOT_FILE_PREFIX + "." + Long.toHexString(zxid); } /** @@ -157,7 +157,7 @@ public class Util { * @throws IOException */ public static boolean isValidSnapshot(File f) throws IOException { - if (f==null || Util.getZxidFromName(f.getName(), "snapshot") == -1) + if (f==null || Util.getZxidFromName(f.getName(), FileSnap.SNAPSHOT_FILE_PREFIX) == -1) return false; // Check for a valid snapshot @@ -297,5 +297,25 @@ public class Util { Collections.sort(filelist, new DataDirFileComparator(prefix, ascending)); return filelist; } + + /** + * Returns true if fileName is a log file name. + * + * @param fileName + * @return + */ + public static boolean isLogFileName(String fileName) { + return fileName.startsWith(FileTxnLog.LOG_FILE_PREFIX + "."); + } + + /** + * Returns true if fileName is a snapshot file name. + * + * @param fileName + * @return + */ + public static boolean isSnapshotFileName(String fileName) { + return fileName.startsWith(FileSnap.SNAPSHOT_FILE_PREFIX + "."); + } } http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/test/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java ---------------------------------------------------------------------- diff --git a/src/java/test/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java b/src/java/test/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java new file mode 100644 index 0000000..0c2da9c --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java @@ -0,0 +1,148 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server.persistence; + +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; + +public class FileTxnSnapLogTest { + + private File tmpDir; + + private File logDir; + + private File snapDir; + + private File logVersionDir; + + private File snapVersionDir; + + @Before + public void setUp() throws Exception { + tmpDir = ClientBase.createEmptyTestDir(); + logDir = new File(tmpDir, "logdir"); + snapDir = new File(tmpDir, "snapdir"); + } + + @After + public void tearDown() throws Exception { + if(tmpDir != null){ + TestUtils.deleteFileRecursively(tmpDir); + } + this.tmpDir = null; + this.logDir = null; + this.snapDir = null; + this.logVersionDir = null; + this.snapVersionDir = null; + } + + private File createVersionDir(File parentDir) { + File versionDir = new File(parentDir, FileTxnSnapLog.version + FileTxnSnapLog.VERSION); + versionDir.mkdirs(); + return versionDir; + } + + private void createLogFile(File dir, long zxid) throws IOException { + File file = new File(dir.getPath() + File.separator + Util.makeLogName(zxid)); + file.createNewFile(); + } + + private void createSnapshotFile(File dir, long zxid) throws IOException { + File file = new File(dir.getPath() + File.separator + Util.makeSnapshotName(zxid)); + file.createNewFile(); + } + + private void twoDirSetupWithCorrectFiles() throws IOException { + logVersionDir = createVersionDir(logDir); + snapVersionDir = createVersionDir(snapDir); + + // transaction log files in log dir + createLogFile(logVersionDir,1); + createLogFile(logVersionDir,2); + + // snapshot files in snap dir + createSnapshotFile(snapVersionDir,1); + createSnapshotFile(snapVersionDir,2); + } + + private void singleDirSetupWithCorrectFiles() throws IOException { + logVersionDir = createVersionDir(logDir); + + // transaction log and snapshot files in the same dir + createLogFile(logVersionDir,1); + createLogFile(logVersionDir,2); + createSnapshotFile(logVersionDir,1); + createSnapshotFile(logVersionDir,2); + } + + @Test + public void testDirCheckWithCorrectFiles() throws IOException { + twoDirSetupWithCorrectFiles(); + + try { + new FileTxnSnapLog(logDir, snapDir); + } catch (FileTxnSnapLog.LogDirContentCheckException e) { + Assert.fail("Should not throw LogDirContentCheckException."); + } catch ( FileTxnSnapLog.SnapDirContentCheckException e){ + Assert.fail("Should not throw SnapDirContentCheckException."); + } + } + + @Test + public void testDirCheckWithSingleDirSetup() throws IOException { + singleDirSetupWithCorrectFiles(); + + try { + new FileTxnSnapLog(logDir, logDir); + } catch (FileTxnSnapLog.LogDirContentCheckException e) { + Assert.fail("Should not throw LogDirContentCheckException."); + } catch ( FileTxnSnapLog.SnapDirContentCheckException e){ + Assert.fail("Should not throw SnapDirContentCheckException."); + } + } + + @Test(expected = FileTxnSnapLog.LogDirContentCheckException.class) + public void testDirCheckWithSnapFilesInLogDir() throws IOException { + twoDirSetupWithCorrectFiles(); + + // add snapshot files to the log version dir + createSnapshotFile(logVersionDir,3); + createSnapshotFile(logVersionDir,4); + + new FileTxnSnapLog(logDir, snapDir); + } + + @Test(expected = FileTxnSnapLog.SnapDirContentCheckException.class) + public void testDirCheckWithLogFilesInSnapDir() throws IOException { + twoDirSetupWithCorrectFiles(); + + // add transaction log files to the snap version dir + createLogFile(snapVersionDir,3); + createLogFile(snapVersionDir,4); + + new FileTxnSnapLog(logDir, snapDir); + } +} http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/test/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java ---------------------------------------------------------------------- diff --git a/src/java/test/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java b/src/java/test/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java index fe86f54..cbd2b77 100644 --- a/src/java/test/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java +++ b/src/java/test/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java @@ -43,7 +43,7 @@ public class AtomicFileOutputStreamTest extends ZKTestCase { @Before public void setupTestDir() throws IOException { - testDir = ClientBase.createTmpDir(); + testDir = ClientBase.createEmptyTestDir(); dstFile = new File(testDir, "test.txt"); } @After http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/test/org/apache/zookeeper/test/ClientBase.java ---------------------------------------------------------------------- diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java index 7595a02..74d0eed 100644 --- a/src/java/test/org/apache/zookeeper/test/ClientBase.java +++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java @@ -351,11 +351,15 @@ public abstract class ClientBase extends ZKTestCase { } } + public static File createEmptyTestDir() throws IOException { + return createTmpDir(BASETEST, false); + } public static File createTmpDir() throws IOException { - return createTmpDir(BASETEST); + return createTmpDir(BASETEST, true); } - static File createTmpDir(File parentDir) throws IOException { + + static File createTmpDir(File parentDir, boolean createInitFile) throws IOException { File tmpFile = File.createTempFile("test", ".junit", parentDir); // don't delete tmpFile - this ensures we don't attempt to create // a tmpDir with a duplicate name @@ -363,8 +367,21 @@ public abstract class ClientBase extends ZKTestCase { Assert.assertFalse(tmpDir.exists()); // never true if tmpfile does it's job Assert.assertTrue(tmpDir.mkdirs()); + // todo not every tmp directory needs this file + if (createInitFile) { + createInitializeFile(tmpDir); + } + return tmpDir; } + + public static void createInitializeFile(File dir) throws IOException { + File initFile = new File(dir, "initialize"); + if (!initFile.exists()) { + Assert.assertTrue(initFile.createNewFile()); + } + } + private static int getPort(String hostPort) { String[] split = hostPort.split(":"); String portstr = split[split.length-1]; @@ -478,7 +495,7 @@ public abstract class ClientBase extends ZKTestCase { setUpAll(); - tmpDir = createTmpDir(BASETEST); + tmpDir = createTmpDir(BASETEST, true); startServer(); http://git-wip-us.apache.org/repos/asf/zookeeper/blob/997fa5e7/src/java/test/org/apache/zookeeper/test/TestUtils.java ---------------------------------------------------------------------- diff --git a/src/java/test/org/apache/zookeeper/test/TestUtils.java b/src/java/test/org/apache/zookeeper/test/TestUtils.java new file mode 100644 index 0000000..fa08b82 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/TestUtils.java @@ -0,0 +1,60 @@ +/** + * 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.zookeeper.test; + +import java.io.File; + +import org.junit.Assert; + +/** + * This class contains test utility methods + */ +public class TestUtils { + + /** + * deletes a folder recursively + * + * @param file + * folder to be deleted + * @param failOnError + * if true file deletion success is ensured + */ + public static boolean deleteFileRecursively(File file, + final boolean failOnError) { + if (file != null) { + if (file.isDirectory()) { + File[] files = file.listFiles(); + int size = files.length; + for (int i = 0; i < size; i++) { + File f = files[i]; + boolean deleted = deleteFileRecursively(files[i], failOnError); + if(!deleted && failOnError) + { + Assert.fail("file '" + f.getAbsolutePath()+"' deletion failed"); + } + } + } + return file.delete(); + } + return true; + } + + public static boolean deleteFileRecursively(File file) { + return deleteFileRecursively(file, false); + } +}