-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-28010: Prevent connection attribute corruption #5366
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
a99d6b9
8de34c2
953a28b
a665364
4fa6c4a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,16 +18,19 @@ | |
| package org.apache.hadoop.hbase.ipc; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Map; | ||
| import java.util.concurrent.ExecutorService; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.hbase.CellScanner; | ||
| import org.apache.hadoop.hbase.HBaseInterfaceAudience; | ||
| import org.apache.hadoop.hbase.security.User; | ||
| import org.apache.yetus.audience.InterfaceAudience; | ||
| import org.apache.yetus.audience.InterfaceStability; | ||
|
|
||
| import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService; | ||
| import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; | ||
| import org.apache.hbase.thirdparty.com.google.protobuf.Message; | ||
|
|
||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; | ||
|
|
||
| /** | ||
|
|
@@ -83,7 +86,17 @@ public interface RpcCall extends RpcCallContext { | |
| /** Returns The request header of this call. */ | ||
| RequestHeader getHeader(); | ||
|
|
||
| ConnectionHeader getConnectionHeader(); | ||
| /** | ||
| * Returns the map of attributes specified when building the Connection See the Map argument on | ||
|
||
| * {@link org.apache.hadoop.hbase.client.ConnectionFactory#createConnection(Configuration, ExecutorService, User, Map)} | ||
| */ | ||
| Map<String, byte[]> getConnectionAttributes(); | ||
|
|
||
| /** | ||
| * Returns the map of attributes specified when building the request See | ||
| * {@link org.apache.hadoop.hbase.client.TableBuilder#setRequestAttribute(String, byte[])} | ||
| */ | ||
| Map<String, byte[]> getRequestAttributes(); | ||
|
|
||
| /** Returns Port of remote address in this call */ | ||
| int getRemotePort(); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,9 +24,12 @@ | |
| import java.net.InetAddress; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Optional; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import org.apache.curator.shaded.com.google.common.collect.Maps; | ||
| import org.apache.hadoop.hbase.CellScanner; | ||
| import org.apache.hadoop.hbase.DoNotRetryIOException; | ||
| import org.apache.hadoop.hbase.HBaseServerException; | ||
|
|
@@ -48,8 +51,8 @@ | |
| import org.apache.hbase.thirdparty.com.google.protobuf.Message; | ||
|
|
||
| import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; | ||
|
|
@@ -99,6 +102,7 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa | |
| // cumulative size of serialized exceptions | ||
| private long exceptionSize = 0; | ||
| private final boolean retryImmediatelySupported; | ||
| private Map<String, byte[]> requestAttributes; | ||
|
|
||
| // This is a dirty hack to address HBASE-22539. The highest bit is for rpc ref and cleanup, and | ||
| // the rest of the bits are for WAL reference count. We can only call release if all of them are | ||
|
|
@@ -209,8 +213,24 @@ public RequestHeader getHeader() { | |
| } | ||
|
|
||
| @Override | ||
| public RPCProtos.ConnectionHeader getConnectionHeader() { | ||
| return this.connection.connectionHeader; | ||
| public Map<String, byte[]> getConnectionAttributes() { | ||
| return this.connection.connectionAttributes; | ||
| } | ||
|
|
||
| @Override | ||
| public Map<String, byte[]> getRequestAttributes() { | ||
| if (this.requestAttributes == null) { | ||
| if (header.getAttributeList().isEmpty()) { | ||
| this.requestAttributes = Collections.emptyMap(); | ||
| } else { | ||
| this.requestAttributes = Maps.newHashMapWithExpectedSize(header.getAttributeList().size()); | ||
|
||
| for (HBaseProtos.NameBytesPair nameBytesPair : header.getAttributeList()) { | ||
| this.requestAttributes.put(nameBytesPair.getName(), | ||
| nameBytesPair.getValue().toByteArray()); | ||
| } | ||
| } | ||
| } | ||
| return this.requestAttributes; | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,11 +31,14 @@ | |
| import java.net.InetSocketAddress; | ||
| import java.nio.ByteBuffer; | ||
| import java.security.GeneralSecurityException; | ||
| import java.util.Collections; | ||
| import java.util.Map; | ||
| import java.util.Objects; | ||
| import java.util.Properties; | ||
| import org.apache.commons.crypto.cipher.CryptoCipherFactory; | ||
| import org.apache.commons.crypto.random.CryptoRandom; | ||
| import org.apache.commons.crypto.random.CryptoRandomFactory; | ||
| import org.apache.curator.shaded.com.google.common.collect.Maps; | ||
| import org.apache.hadoop.hbase.CellScanner; | ||
| import org.apache.hadoop.hbase.DoNotRetryIOException; | ||
| import org.apache.hadoop.hbase.client.VersionInfoUtil; | ||
|
|
@@ -75,6 +78,7 @@ | |
| import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; | ||
|
|
||
| import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; | ||
| import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; | ||
|
|
@@ -103,6 +107,7 @@ abstract class ServerRpcConnection implements Closeable { | |
| protected int remotePort; | ||
| protected InetAddress addr; | ||
| protected ConnectionHeader connectionHeader; | ||
| protected Map<String, byte[]> connectionAttributes; | ||
|
|
||
| /** | ||
| * Codec the client asked use. | ||
|
|
@@ -405,6 +410,16 @@ private CodedInputStream createCis(ByteBuff buf) { | |
| // Reads the connection header following version | ||
| private void processConnectionHeader(ByteBuff buf) throws IOException { | ||
| this.connectionHeader = ConnectionHeader.parseFrom(createCis(buf)); | ||
| if (connectionHeader.getAttributeList().isEmpty()) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you add a comment here that we want to copy the attributes prior to releasing the buffer so that they don't get corrupted down the line when the buffer's underlying memory is replaced for some other call? |
||
| this.connectionAttributes = Collections.emptyMap(); | ||
| } else { | ||
| this.connectionAttributes = | ||
| Maps.newHashMapWithExpectedSize(connectionHeader.getAttributeList().size()); | ||
| for (HBaseProtos.NameBytesPair nameBytesPair : connectionHeader.getAttributeList()) { | ||
| this.connectionAttributes.put(nameBytesPair.getName(), | ||
| nameBytesPair.getValue().toByteArray()); | ||
| } | ||
| } | ||
| String serviceName = connectionHeader.getServiceName(); | ||
| if (serviceName == null) { | ||
| throw new EmptyServiceNameException(); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.