You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/06/08 08:40:32 UTC

[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 <el...@apache.org>
Authored: Sun Jun 8 02:07:27 2014 -0400
Committer: Josh Elser <el...@apache.org>
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<Key,Value> 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.<Entry<String,String>> 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());
+  }
+
+}