Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-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 66195102A0 for ; Sat, 3 Aug 2013 20:39:06 +0000 (UTC) Received: (qmail 3537 invoked by uid 500); 3 Aug 2013 20:39:06 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 3501 invoked by uid 500); 3 Aug 2013 20:39:06 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 3494 invoked by uid 99); 3 Aug 2013 20:39:06 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 03 Aug 2013 20:39:06 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 03 Aug 2013 20:38:59 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id A21EF23889EC; Sat, 3 Aug 2013 20:38:37 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1510066 - /hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java Date: Sat, 03 Aug 2013 20:38:37 -0000 To: commits@hbase.apache.org From: stack@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20130803203837.A21EF23889EC@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: stack Date: Sat Aug 3 20:38:37 2013 New Revision: 1510066 URL: http://svn.apache.org/r1510066 Log: HBASE-8978 Restore TestLogRollAbort upon review Added: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java Added: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java?rev=1510066&view=auto ============================================================================== --- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java (added) +++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java Sat Aug 3 20:38:37 2013 @@ -0,0 +1,165 @@ +/** + * 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.hadoop.hbase.regionserver.wal; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.impl.Log4JLogger; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.LeaseManager; +import org.apache.log4j.Level; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Tests for conditions that should trigger RegionServer aborts when + * rolling the current HLog fails. + */ +@Category(MediumTests.class) +public class TestLogRollAbort { + private static final Log LOG = LogFactory.getLog(TestLogRolling.class); + private static MiniDFSCluster dfsCluster; + private static HBaseAdmin admin; + private static MiniHBaseCluster cluster; + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + // verbose logging on classes that are touched in these tests + { + ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem")) + .getLogger().setLevel(Level.ALL); + ((Log4JLogger)HRegionServer.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)HRegion.LOG).getLogger().setLevel(Level.ALL); + ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL); + } + + // Need to override this setup so we can edit the config before it gets sent + // to the HDFS & HBase cluster startup. + @BeforeClass + public static void setUpBeforeClass() throws Exception { + // Tweak default timeout values down for faster recovery + TEST_UTIL.getConfiguration().setInt( + "hbase.regionserver.logroll.errors.tolerated", 2); + TEST_UTIL.getConfiguration().setInt("ipc.ping.interval", 10 * 1000); + TEST_UTIL.getConfiguration().setInt("ipc.socket.timeout", 10 * 1000); + TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000); + + // Increase the amount of time between client retries + TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 5 * 1000); + + // make sure log.hflush() calls syncFs() to open a pipeline + TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true); + // lower the namenode & datanode heartbeat so the namenode + // quickly detects datanode failures + TEST_UTIL.getConfiguration().setInt("heartbeat.recheck.interval", 5000); + TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1); + // the namenode might still try to choose the recently-dead datanode + // for a pipeline, so try to a new pipeline multiple times + TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 10); + // set periodic sync to 2 min so it doesn't run during test + TEST_UTIL.getConfiguration().setInt("hbase.regionserver.optionallogflushinterval", + 120 * 1000); + } + + @Before + public void setUp() throws Exception { + TEST_UTIL.startMiniCluster(2); + + cluster = TEST_UTIL.getHBaseCluster(); + dfsCluster = TEST_UTIL.getDFSCluster(); + admin = TEST_UTIL.getHBaseAdmin(); + + // disable region rebalancing (interferes with log watching) + cluster.getMaster().balanceSwitch(false); + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * Tests that RegionServer aborts if we hit an error closing the WAL when + * there are unsynced WAL edits. See HBASE-4282. + */ + @Test + public void testRSAbortWithUnflushedEdits() throws Exception { + LOG.info("Starting testRSAbortWithUnflushedEdits()"); + + // When the META table can be opened, the region servers are running + new HTable(TEST_UTIL.getConfiguration(), + HConstants.META_TABLE_NAME).close(); + + // Create the test table and open it + String tableName = this.getClass().getSimpleName(); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + desc.setDeferredLogFlush(true); + + admin.createTable(desc); + HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + + HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName)); + HLog log = server.getWAL(); + + assertTrue("Need HDFS-826 for this test", ((FSHLog) log).canGetCurReplicas()); + // don't run this test without append support (HDFS-200 & HDFS-142) + assertTrue("Need append support for this test", + FSUtils.isAppendSupported(TEST_UTIL.getConfiguration())); + + Put p = new Put(Bytes.toBytes("row2001")); + p.add(HConstants.CATALOG_FAMILY, Bytes.toBytes("col"), Bytes.toBytes(2001)); + table.put(p); + + log.sync(); + + p = new Put(Bytes.toBytes("row2002")); + p.add(HConstants.CATALOG_FAMILY, Bytes.toBytes("col"), Bytes.toBytes(2002)); + table.put(p); + + dfsCluster.restartDataNodes(); + LOG.info("Restarted datanodes"); + + assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries()); + try { + log.rollWriter(true); + fail("Log roll should have triggered FailedLogCloseException"); + } catch (FailedLogCloseException flce) { + assertTrue("Should have deferred flush log edits outstanding", + ((FSHLog) log).hasDeferredEntries()); + } + } + +} +