Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
HBASE-15077 Support OffheapKV write in compaction with out copying da…
…ta on heap.
- Loading branch information
1 parent
ec47a81
commit da932ee
Showing
10 changed files
with
245 additions
and
55 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
129 changes: 129 additions & 0 deletions
129
hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteArrayOutputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,129 @@ | ||
/** | ||
* 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; | ||
|
||
import java.io.IOException; | ||
import java.io.OutputStream; | ||
import java.nio.BufferOverflowException; | ||
import java.nio.ByteBuffer; | ||
import java.util.Arrays; | ||
|
||
import org.apache.hadoop.hbase.classification.InterfaceAudience; | ||
import org.apache.hadoop.hbase.util.ByteBufferUtils; | ||
import org.apache.hadoop.hbase.util.Bytes; | ||
|
||
/** | ||
* Our own implementation of ByteArrayOutputStream where all methods are NOT synchronized and | ||
* supports writing ByteBuffer directly to it. | ||
*/ | ||
@InterfaceAudience.Private | ||
public class ByteArrayOutputStream extends OutputStream implements ByteBufferSupportOutputStream { | ||
|
||
// Borrowed from openJDK: | ||
// http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/8-b132/java/util/ArrayList.java#221 | ||
private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8; | ||
|
||
private byte[] buf; | ||
private int pos = 0; | ||
|
||
public ByteArrayOutputStream() { | ||
this(32); | ||
} | ||
|
||
public ByteArrayOutputStream(int capacity) { | ||
this.buf = new byte[capacity]; | ||
} | ||
|
||
@Override | ||
public void write(ByteBuffer b, int off, int len) throws IOException { | ||
checkSizeAndGrow(len); | ||
ByteBufferUtils.copyFromBufferToArray(this.buf, b, off, this.pos, len); | ||
this.pos += len; | ||
} | ||
|
||
@Override | ||
public void writeInt(int i) throws IOException { | ||
checkSizeAndGrow(Bytes.SIZEOF_INT); | ||
Bytes.putInt(this.buf, this.pos, i); | ||
this.pos += Bytes.SIZEOF_INT; | ||
} | ||
|
||
@Override | ||
public void write(int b) throws IOException { | ||
checkSizeAndGrow(Bytes.SIZEOF_BYTE); | ||
buf[this.pos] = (byte) b; | ||
this.pos++; | ||
} | ||
|
||
@Override | ||
public void write(byte[] b, int off, int len) throws IOException { | ||
checkSizeAndGrow(len); | ||
System.arraycopy(b, off, this.buf, this.pos, len); | ||
this.pos += len; | ||
} | ||
|
||
private void checkSizeAndGrow(int extra) { | ||
long capacityNeeded = this.pos + (long) extra; | ||
if (capacityNeeded > this.buf.length) { | ||
// guarantee it's possible to fit | ||
if (capacityNeeded > MAX_ARRAY_SIZE) { | ||
throw new BufferOverflowException(); | ||
} | ||
// double until hit the cap | ||
long nextCapacity = Math.min(this.buf.length << 1, MAX_ARRAY_SIZE); | ||
// but make sure there is enough if twice the existing capacity is still too small | ||
nextCapacity = Math.max(nextCapacity, capacityNeeded); | ||
if (nextCapacity > MAX_ARRAY_SIZE) { | ||
throw new BufferOverflowException(); | ||
} | ||
byte[] newBuf = new byte[(int) nextCapacity]; | ||
System.arraycopy(buf, 0, newBuf, 0, buf.length); | ||
buf = newBuf; | ||
} | ||
} | ||
|
||
/** | ||
* Resets the <code>pos</code> field of this byte array output stream to zero. The output stream | ||
* can be used again. | ||
*/ | ||
public void reset() { | ||
this.pos = 0; | ||
} | ||
|
||
/** | ||
* Copies the content of this Stream into a new byte array. | ||
* @return the contents of this output stream, as new byte array. | ||
*/ | ||
public byte toByteArray()[] { | ||
return Arrays.copyOf(buf, pos); | ||
} | ||
|
||
/** | ||
* @return the underlying array where the data gets accumulated | ||
*/ | ||
public byte[] getBuffer() { | ||
return this.buf; | ||
} | ||
|
||
/** | ||
* @return The current size of the buffer. | ||
*/ | ||
public int size() { | ||
return this.pos; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
44 changes: 44 additions & 0 deletions
44
hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportDataOutputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/** | ||
* 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; | ||
|
||
import java.io.DataOutputStream; | ||
import java.io.IOException; | ||
import java.io.OutputStream; | ||
import java.nio.ByteBuffer; | ||
|
||
import org.apache.hadoop.hbase.classification.InterfaceAudience; | ||
import org.apache.hadoop.hbase.util.ByteBufferUtils; | ||
|
||
/** | ||
* Our extension of DataOutputStream which implements ByteBufferSupportOutputStream | ||
*/ | ||
@InterfaceAudience.Private | ||
public class ByteBufferSupportDataOutputStream extends DataOutputStream | ||
implements ByteBufferSupportOutputStream { | ||
|
||
public ByteBufferSupportDataOutputStream(OutputStream out) { | ||
super(out); | ||
} | ||
|
||
@Override | ||
public void write(ByteBuffer b, int off, int len) throws IOException { | ||
ByteBufferUtils.copyBufferToStream(out, b, off, len); | ||
written += len; | ||
} | ||
} |
51 changes: 51 additions & 0 deletions
51
hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferSupportOutputStream.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,51 @@ | ||
/** | ||
* 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; | ||
|
||
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
|
||
import org.apache.hadoop.hbase.classification.InterfaceAudience; | ||
|
||
/** | ||
* Interface adds support for writing {@link ByteBuffer} into OutputStream. | ||
*/ | ||
@InterfaceAudience.Private | ||
public interface ByteBufferSupportOutputStream { | ||
|
||
/** | ||
* Writes <code>len</code> bytes from the specified ByteBuffer starting at offset <code>off</code> | ||
* to this output stream. | ||
* | ||
* @param b the data. | ||
* @param off the start offset in the data. | ||
* @param len the number of bytes to write. | ||
* @exception IOException | ||
* if an I/O error occurs. In particular, an <code>IOException</code> is thrown if | ||
* the output stream is closed. | ||
*/ | ||
void write(ByteBuffer b, int off, int len) throws IOException; | ||
|
||
/** | ||
* Writes an <code>int</code> to the underlying output stream as four | ||
* bytes, high byte first. | ||
* @param i the <code>int</code> to write | ||
* @throws IOException if an I/O error occurs. | ||
*/ | ||
void writeInt(int i) throws IOException; | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.