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 B59321156C for ; Sun, 8 Jun 2014 06:40:32 +0000 (UTC) Received: (qmail 3977 invoked by uid 500); 8 Jun 2014 06:40:32 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 3861 invoked by uid 500); 8 Jun 2014 06:40:32 -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 3847 invoked by uid 99); 8 Jun 2014 06:40:32 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 08 Jun 2014 06:40:32 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 0F34C467EA; Sun, 8 Jun 2014 06:40:32 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: elserj@apache.org To: commits@accumulo.apache.org Date: Sun, 08 Jun 2014 06:40:32 -0000 Message-Id: <962283f41da14f8fa306833918839b5b@git.apache.org> In-Reply-To: <504e31c623bd42d9aad0dbe6f914d781@git.apache.org> References: <504e31c623bd42d9aad0dbe6f914d781@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/6] git commit: ACCUMULO-2869 Ensure that the proper property is set for the index block size. ACCUMULO-2869 Ensure that the proper property is set for the index block size. Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7a8ec735 Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7a8ec735 Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7a8ec735 Branch: refs/heads/master Commit: 7a8ec735be8fd6bb91036f6cc9cc91ec5c50efba Parents: 18d6ca1 Author: Josh Elser Authored: Sun Jun 8 02:07:27 2014 -0400 Committer: Josh Elser Committed: Sun Jun 8 02:07:27 2014 -0400 ---------------------------------------------------------------------- .../org/apache/accumulo/tserver/Tablet.java | 18 ++++- .../org/apache/accumulo/tserver/TabletTest.java | 75 ++++++++++++++++++++ 2 files changed, 91 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a8ec735/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java index 3fe60b7..f73d4ca 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java @@ -151,6 +151,8 @@ import org.apache.log4j.Logger; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; +import com.google.common.annotations.VisibleForTesting; + /* * We need to be able to have the master tell a tabletServer to * close this file, and the tablet server to handle all pending client reads @@ -1062,6 +1064,18 @@ public class Tablet { splitCreationTime = System.currentTimeMillis(); } + /** + * Only visibile for testing + */ + @VisibleForTesting + protected Tablet(TabletTime tabletTime, String tabletDirectory, int logId, Path location, DatafileManager datafileManager) { + this.tabletTime = tabletTime; + this.tabletDirectory = tabletDirectory; + this.logId = logId; + this.location = location; + this.datafileManager = datafileManager; + } + private Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm, Configuration conf, SortedMap tabletsKeyValues) throws IOException { this(tabletServer, location, extent, trm, conf, VolumeManagerImpl.get(), tabletsKeyValues); @@ -3228,7 +3242,7 @@ public class Tablet { } } - private AccumuloConfiguration createTableConfiguration(TableConfiguration base, CompactionPlan plan) { + protected AccumuloConfiguration createTableConfiguration(TableConfiguration base, CompactionPlan plan) { if (plan == null || plan.writeParameters == null) return base; WriteParameters p = plan.writeParameters; @@ -3238,7 +3252,7 @@ public class Tablet { if (p.getBlockSize() > 0) result.set(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, "" + p.getBlockSize()); if (p.getIndexBlockSize() > 0) - result.set(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, "" + p.getBlockSize()); + result.set(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, "" + p.getIndexBlockSize()); if (p.getCompressType() != null) result.set(Property.TABLE_FILE_COMPRESSION_TYPE, p.getCompressType()); if (p.getReplication() != 0) http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a8ec735/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletTest.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletTest.java new file mode 100644 index 0000000..cbd6689 --- /dev/null +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletTest.java @@ -0,0 +1,75 @@ +/* + * 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; + +import java.util.Map.Entry; + +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.server.conf.TableConfiguration; +import org.apache.accumulo.server.tablets.TabletTime; +import org.apache.accumulo.tserver.Tablet.DatafileManager; +import org.apache.accumulo.tserver.compaction.CompactionPlan; +import org.apache.accumulo.tserver.compaction.WriteParameters; +import org.apache.hadoop.fs.Path; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +import com.google.common.collect.Iterators; + +/** + * + */ +public class TabletTest { + + @Test + public void correctValuesSetForProperties() { + TableConfiguration tableConf = EasyMock.createMock(TableConfiguration.class); + CompactionPlan plan = EasyMock.createMock(CompactionPlan.class); + WriteParameters writeParams = EasyMock.createMock(WriteParameters.class); + plan.writeParameters = writeParams; + DatafileManager dfm = EasyMock.createMock(DatafileManager.class); + TabletTime time = EasyMock.createMock(TabletTime.class); + + Tablet tablet = new Tablet(time, "", 0, new Path("/foo"), dfm); + + long hdfsBlockSize = 10000l, blockSize = 5000l, indexBlockSize = 500l; + int replication = 5; + String compressType = "snappy"; + + EasyMock.expect(tableConf.iterator()).andReturn(Iterators.> emptyIterator()); + EasyMock.expect(writeParams.getHdfsBlockSize()).andReturn(hdfsBlockSize).times(2); + EasyMock.expect(writeParams.getBlockSize()).andReturn(blockSize).times(2); + EasyMock.expect(writeParams.getIndexBlockSize()).andReturn(indexBlockSize).times(2); + EasyMock.expect(writeParams.getCompressType()).andReturn(compressType).times(2); + EasyMock.expect(writeParams.getReplication()).andReturn(replication).times(2); + + EasyMock.replay(tableConf, plan, writeParams); + + AccumuloConfiguration aConf = tablet.createTableConfiguration(tableConf, plan); + + EasyMock.verify(tableConf, plan, writeParams); + + Assert.assertEquals(hdfsBlockSize, Long.valueOf(aConf.get(Property.TABLE_FILE_BLOCK_SIZE)).longValue()); + Assert.assertEquals(blockSize, Long.valueOf(aConf.get(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE)).longValue()); + Assert.assertEquals(indexBlockSize, Long.valueOf(aConf.get(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX)).longValue()); + Assert.assertEquals(compressType, aConf.get(Property.TABLE_FILE_COMPRESSION_TYPE)); + Assert.assertEquals(replication, Integer.valueOf(aConf.get(Property.TABLE_FILE_REPLICATION)).intValue()); + } + +}