You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2019/03/06 17:33:59 UTC

[hbase] branch branch-2.1 updated: Revert oHBASE-21874 Bucket cache on Persistent memory"

This is an automated email from the ASF dual-hosted git repository.

ramkrishna pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.1 by this push:
     new e5fb821  Revert oHBASE-21874 Bucket cache on Persistent memory"
e5fb821 is described below

commit e5fb8214b2bfd6396539a4e8b6cf5f3cc5e9c06f
Author: ramkrishna <ra...@apache.org>
AuthorDate: Wed Mar 6 22:57:57 2019 +0530

    Revert oHBASE-21874 Bucket cache on Persistent memory"
    
    This reverts commit 8f8323403baddd1e5fc31e04c79a99ee74a934bd.
---
 hbase-common/src/main/resources/hbase-default.xml  |  6 +-
 .../hadoop/hbase/io/hfile/bucket/BucketCache.java  | 10 +--
 .../hfile/bucket/ExclusiveMemoryMmapIOEngine.java  | 50 ---------------
 .../{FileMmapIOEngine.java => FileMmapEngine.java} | 73 ++++++++++++----------
 .../io/hfile/bucket/SharedMemoryMmapIOEngine.java  | 64 -------------------
 ...moryMmapEngine.java => TestFileMmapEngine.java} |  8 +--
 src/main/asciidoc/_chapters/architecture.adoc      |  2 -
 src/main/asciidoc/_chapters/hbase-default.adoc     | 11 ++--
 8 files changed, 50 insertions(+), 174 deletions(-)

diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 4c74748..d7e4476 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -913,10 +913,8 @@ possible configurations would overwhelm and obscure the important.
     <name>hbase.bucketcache.ioengine</name>
     <value></value>
     <description>Where to store the contents of the bucketcache. One of: offheap,
-    file, files, mmap or pmem. If a file or files, set it to file(s):PATH_TO_FILE.
-    mmap means the content will be in an mmaped file. Use mmap:PATH_TO_FILE. 'pmem'
-    is bucket cache over a file on the persistent memory device.
-    Use pmem:PATH_TO_FILE.
+    file, files or mmap. If a file or files, set it to file(s):PATH_TO_FILE.
+    mmap means the content will be in an mmaped file. Use mmap:PATH_TO_FILE.
     See http://hbase.apache.org/book.html#offheap.blockcache for more information.
     </description>
   </property>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index a0a88a7..333b7ef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -383,15 +383,7 @@ public class BucketCache implements BlockCache, HeapSize {
     } else if (ioEngineName.startsWith("offheap")) {
       return new ByteBufferIOEngine(capacity);
     } else if (ioEngineName.startsWith("mmap:")) {
-      return new ExclusiveMemoryMmapIOEngine(ioEngineName.substring(5), capacity);
-    } else if (ioEngineName.startsWith("pmem:")) {
-      // This mode of bucket cache creates an IOEngine over a file on the persistent memory
-      // device. Since the persistent memory device has its own address space the contents
-      // mapped to this address space does not get swapped out like in the case of mmapping
-      // on to DRAM. Hence the cells created out of the hfile blocks in the pmem bucket cache
-      // can be directly referred to without having to copy them onheap. Once the RPC is done,
-      // the blocks can be returned back as in case of ByteBufferIOEngine.
-      return new SharedMemoryMmapIOEngine(ioEngineName.substring(5), capacity);
+      return new FileMmapEngine(ioEngineName.substring(5), capacity);
     } else {
       throw new IllegalArgumentException(
           "Don't understand io engine name for cache- prefix with file:, files:, mmap: or offheap");
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ExclusiveMemoryMmapIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ExclusiveMemoryMmapIOEngine.java
deleted file mode 100644
index 8b024f0..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ExclusiveMemoryMmapIOEngine.java
+++ /dev/null
@@ -1,50 +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.hadoop.hbase.io.hfile.bucket;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.io.hfile.Cacheable;
-import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
-import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
-import org.apache.hadoop.hbase.nio.SingleByteBuff;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * IO engine that stores data to a file on the local block device using memory mapping
- * mechanism
- */
-@InterfaceAudience.Private
-public class ExclusiveMemoryMmapIOEngine extends FileMmapIOEngine {
-  static final Logger LOG = LoggerFactory.getLogger(ExclusiveMemoryMmapIOEngine.class);
-
-  public ExclusiveMemoryMmapIOEngine(String filePath, long capacity) throws IOException {
-    super(filePath, capacity);
-  }
-
-  @Override
-  public Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
-      throws IOException {
-    byte[] dst = new byte[length];
-    bufferArray.getMultiple(offset, length, dst);
-    return deserializer.deserialize(new SingleByteBuff(ByteBuffer.wrap(dst)), true,
-      MemoryType.EXCLUSIVE);
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
similarity index 66%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapIOEngine.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
index 9580efe..82f42cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
@@ -1,19 +1,20 @@
 /**
- * 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
+ * Copyright The Apache Software Foundation
  *
- *     http://www.apache.org/licenses/LICENSE-2.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.
+ * 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.io.hfile.bucket;
 
@@ -23,31 +24,33 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.io.hfile.Cacheable;
+import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
 import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
 import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.util.ByteBufferAllocator;
 import org.apache.hadoop.hbase.util.ByteBufferArray;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
- * IO engine that stores data to a file on the specified file system using memory mapping
+ * IO engine that stores data to a file on the local file system using memory mapping
  * mechanism
  */
 @InterfaceAudience.Private
-public abstract class FileMmapIOEngine implements IOEngine {
-  static final Logger LOG = LoggerFactory.getLogger(FileMmapIOEngine.class);
+public class FileMmapEngine implements IOEngine {
+  static final Logger LOG = LoggerFactory.getLogger(FileMmapEngine.class);
 
-  protected final String path;
-  protected long size;
-  protected ByteBufferArray bufferArray;
+  private final String path;
+  private long size;
+  private ByteBufferArray bufferArray;
   private final FileChannel fileChannel;
   private RandomAccessFile raf = null;
 
-  public FileMmapIOEngine(String filePath, long capacity) throws IOException {
+  public FileMmapEngine(String filePath, long capacity) throws IOException {
     this.path = filePath;
     this.size = capacity;
     long fileSize = 0;
@@ -61,15 +64,13 @@ public abstract class FileMmapIOEngine implements IOEngine {
       LOG.error("Can't create bucket cache file " + filePath, fex);
       throw fex;
     } catch (IOException ioex) {
-      LOG.error(
-        "Can't extend bucket cache file; insufficient space for " + StringUtils.byteDesc(fileSize),
-        ioex);
+      LOG.error("Can't extend bucket cache file; insufficient space for "
+          + StringUtils.byteDesc(fileSize), ioex);
       shutdown();
       throw ioex;
     }
     ByteBufferAllocator allocator = new ByteBufferAllocator() {
       AtomicInteger pos = new AtomicInteger(0);
-
       @Override
       public ByteBuffer allocate(long size) throws IOException {
         ByteBuffer buffer = fileChannel.map(java.nio.channels.FileChannel.MapMode.READ_WRITE,
@@ -86,8 +87,8 @@ public abstract class FileMmapIOEngine implements IOEngine {
 
   @Override
   public String toString() {
-    return "ioengine=" + this.getClass().getSimpleName() + ", path=" + this.path + ", size="
-        + String.format("%,d", this.size);
+    return "ioengine=" + this.getClass().getSimpleName() + ", path=" + this.path +
+      ", size=" + String.format("%,d", this.size);
   }
 
   /**
@@ -96,13 +97,17 @@ public abstract class FileMmapIOEngine implements IOEngine {
    */
   @Override
   public boolean isPersistent() {
-    // TODO : HBASE-21981 needed for persistence to really work
     return true;
   }
 
   @Override
-  public abstract Cacheable read(long offset, int length,
-      CacheableDeserializer<Cacheable> deserializer) throws IOException;
+  public Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
+      throws IOException {
+    byte[] dst = new byte[length];
+    bufferArray.getMultiple(offset, length, dst);
+    return deserializer.deserialize(new SingleByteBuff(ByteBuffer.wrap(dst)), true,
+      MemoryType.EXCLUSIVE);
+  }
 
   /**
    * Transfers data from the given byte buffer to file
@@ -114,7 +119,7 @@ public abstract class FileMmapIOEngine implements IOEngine {
   public void write(ByteBuffer srcBuffer, long offset) throws IOException {
     assert srcBuffer.hasArray();
     bufferArray.putMultiple(offset, srcBuffer.remaining(), srcBuffer.array(),
-      srcBuffer.arrayOffset());
+        srcBuffer.arrayOffset());
   }
 
   @Override
@@ -122,9 +127,9 @@ public abstract class FileMmapIOEngine implements IOEngine {
     // This singleByteBuff can be considered to be array backed
     assert srcBuffer.hasArray();
     bufferArray.putMultiple(offset, srcBuffer.remaining(), srcBuffer.array(),
-      srcBuffer.arrayOffset());
-  }
+        srcBuffer.arrayOffset());
 
+  }
   /**
    * Sync the data to file after writing
    * @throws IOException
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/SharedMemoryMmapIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/SharedMemoryMmapIOEngine.java
deleted file mode 100644
index b6a7a57..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/SharedMemoryMmapIOEngine.java
+++ /dev/null
@@ -1,64 +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.hadoop.hbase.io.hfile.bucket;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.io.hfile.Cacheable;
-import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
-import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
-import org.apache.hadoop.hbase.nio.ByteBuff;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * IO engine that stores data in pmem devices such as DCPMM. This engine also mmaps the file from
- * the given path. But note that this path has to be a path on the pmem device so that when mmapped
- * the file's address is mapped to the Pmem's address space and not in the DRAM. Since this address
- * space is exclusive for the Pmem device there is no swapping out of the mmapped contents that
- * generally happens when DRAM's free space is not enough to hold the specified file's mmapped
- * contents. This gives us the option of using the {@code MemoryType#SHARED} type when serving the
- * data from this pmem address space. We need not copy the blocks to the onheap space as we need to
- * do for the case of {@code ExclusiveMemoryMmapIOEngine}.
- */
-@InterfaceAudience.Private
-public class SharedMemoryMmapIOEngine extends FileMmapIOEngine {
-
-  // TODO this will support only one path over Pmem. To make use of multiple Pmem devices mounted,
-  // we need to support multiple paths like files IOEngine. Support later.
-  public SharedMemoryMmapIOEngine(String filePath, long capacity) throws IOException {
-    super(filePath, capacity);
-  }
-
-  @Override
-  public boolean usesSharedMemory() {
-    return true;
-  }
-
-  @Override
-  public Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
-      throws IOException {
-    ByteBuff dstBuffer = bufferArray.asSubByteBuff(offset, length);
-    // Here the buffer that is created directly refers to the buffer in the actual buckets.
-    // When any cell is referring to the blocks created out of these buckets then it means that
-    // those cells are referring to a shared memory area which if evicted by the BucketCache would
-    // lead to corruption of results. Hence we set the type of the buffer as SHARED_MEMORY
-    // so that the readers using this block are aware of this fact and do the necessary action
-    // to prevent eviction till the results are either consumed or copied
-    return deserializer.deserialize(dstBuffer, true, MemoryType.SHARED);
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestExclusiveMemoryMmapEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java
similarity index 90%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestExclusiveMemoryMmapEngine.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java
index d0d8c8a..2748d80 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestExclusiveMemoryMmapEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestFileMmapEngine.java
@@ -32,21 +32,21 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
- * Basic test for {@link ExclusiveMemoryMmapIOEngine}
+ * Basic test for {@link FileMmapEngine}
  */
 @Category({IOTests.class, SmallTests.class})
-public class TestExclusiveMemoryMmapEngine {
+public class TestFileMmapEngine {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestExclusiveMemoryMmapEngine.class);
+      HBaseClassTestRule.forClass(TestFileMmapEngine.class);
 
   @Test
   public void testFileMmapEngine() throws IOException {
     int size = 2 * 1024 * 1024; // 2 MB
     String filePath = "testFileMmapEngine";
     try {
-      ExclusiveMemoryMmapIOEngine fileMmapEngine = new ExclusiveMemoryMmapIOEngine(filePath, size);
+      FileMmapEngine fileMmapEngine = new FileMmapEngine(filePath, size);
       for (int i = 0; i < 50; i++) {
         int len = (int) Math.floor(Math.random() * 100);
         long offset = (long) Math.floor(Math.random() * size % (size - len));
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 84476b6..e1905bc 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -733,8 +733,6 @@ See configurations, sizings, current usage, time-in-the-cache, and even detail o
 
 `LruBlockCache` is the original implementation, and is entirely within the Java heap.
 `BucketCache` is optional and mainly intended for keeping block cache data off-heap, although `BucketCache` can also be a file-backed cache.
- In file-backed we can either use it in the file mode or the mmaped mode.
- We also have pmem mode where the bucket cache resides on the persistent memory device.
 
 When you enable BucketCache, you are enabling a two tier caching system. We used to describe the
 tiers as "L1" and "L2" but have deprecated this terminology as of hbase-2.0.0. The "L1" cache referred to an
diff --git a/src/main/asciidoc/_chapters/hbase-default.adoc b/src/main/asciidoc/_chapters/hbase-default.adoc
index ccd5f6f..f809f28 100644
--- a/src/main/asciidoc/_chapters/hbase-default.adoc
+++ b/src/main/asciidoc/_chapters/hbase-default.adoc
@@ -1197,13 +1197,10 @@ When the size of a leaf-level, intermediate-level, or root-level
 *`hbase.bucketcache.ioengine`*::
 +
 .Description
-Where to store the contents of the bucketcache. One of: offheap,
-    file, files, mmap or pmem. If a file or files, set it to file(s):PATH_TO_FILE.
-    mmap means the content will be in an mmaped file. Use mmap:PATH_TO_FILE.
-    'pmem' is bucket cache over a file on the persistent memory device.
-    Use pmem:PATH_TO_FILE.
-    See https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html
-    for more information.
+Where to store the contents of the bucketcache. One of: onheap,
+      offheap, or file. If a file, set it to file:PATH_TO_FILE.
+      See https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html
+      for more information.
 
 +
 .Default