Revert "HBASE-21874 Bucket cache on Persistent memory"

This reverts commit 763202d48e.

bad signed-off-by line
This commit is contained in:
Sean Busbey 2019-03-07 08:49:41 -06:00
parent 415ff2a19d
commit d9793c311c
8 changed files with 53 additions and 177 deletions

View File

@ -925,10 +925,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>

View File

@ -379,15 +379,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");

View File

@ -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);
}
}

View File

@ -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.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.nio.ByteBuff;
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;
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;
/**
* 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

View File

@ -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);
}
}

View File

@ -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));

View File

@ -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

View File

@ -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