Skip to content

Commit

Permalink
DRILL-4280: CORE (bit to bit authentication, data)
Browse files Browse the repository at this point in the history
+ Support authentication in DataServer and DataClient
+ Add AuthenticationCommand as an initial command after handshake
  and before the command that initiates a connection
+ Add DataConnectionConfig to encapsulate configuration
+ Add DataServerRequestHandler to encapsulate all handling of
  requests to DataServer

data
  • Loading branch information
sudheeshkatkam committed Feb 25, 2017
1 parent 180dd56 commit e2582a1
Show file tree
Hide file tree
Showing 11 changed files with 365 additions and 187 deletions.
@@ -1,4 +1,4 @@
/**
/*
* 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
Expand All @@ -17,6 +17,7 @@
*/
package org.apache.drill.exec.rpc.data;

import com.google.common.util.concurrent.SettableFuture;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelFuture;
import io.netty.channel.socket.SocketChannel;
Expand All @@ -30,42 +31,54 @@
import org.apache.drill.exec.rpc.BasicClient;
import org.apache.drill.exec.rpc.OutOfMemoryHandler;
import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.ResponseSender;
import org.apache.drill.exec.rpc.RpcCommand;
import org.apache.drill.exec.rpc.RpcException;
import org.apache.drill.exec.server.BootStrapContext;
import org.apache.drill.exec.rpc.security.AuthenticationOutcomeListener;
import org.apache.drill.exec.rpc.RpcOutcomeListener;

import com.google.protobuf.MessageLite;
import org.apache.hadoop.security.UserGroupInformation;

public class DataClient extends BasicClient<RpcType, DataClientConnection, BitClientHandshake, BitServerHandshake>{
import javax.security.sasl.SaslClient;
import javax.security.sasl.SaslException;
import java.io.IOException;
import java.util.concurrent.ExecutionException;

static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataClient.class);
public class DataClient extends BasicClient<RpcType, DataClientConnection, BitClientHandshake, BitServerHandshake> {
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataClient.class);

private final DrillbitEndpoint remoteEndpoint;
private volatile DataClientConnection connection;
private final BufferAllocator allocator;
private final DataConnectionManager.CloseHandlerCreator closeHandlerFactory;
private final DataConnectionConfig config;


public DataClient(DrillbitEndpoint remoteEndpoint, BootStrapContext context, DataConnectionManager.CloseHandlerCreator closeHandlerFactory) {
public DataClient(DrillbitEndpoint remoteEndpoint, DataConnectionConfig config,
DataConnectionManager.CloseHandlerCreator closeHandlerFactory) {
super(
DataRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
context.getAllocator().getAsByteBufAllocator(),
context.getBitClientLoopGroup(),
DataRpcConfig.getMapping(config.getBootstrapContext().getConfig(),
config.getBootstrapContext().getExecutor()),
config.getBootstrapContext().getAllocator().getAsByteBufAllocator(),
config.getBootstrapContext().getBitClientLoopGroup(),
RpcType.HANDSHAKE,
BitServerHandshake.class,
BitServerHandshake.PARSER);

this.remoteEndpoint = remoteEndpoint;
this.config = config;
this.closeHandlerFactory = closeHandlerFactory;
this.allocator = context.getAllocator();
}

@Override
public DataClientConnection initRemoteConnection(SocketChannel channel) {
protected DataClientConnection initRemoteConnection(SocketChannel channel) {
super.initRemoteConnection(channel);
this.connection = new DataClientConnection(channel, this);
return connection;
}

@Override
protected GenericFutureListener<ChannelFuture> getCloseHandler(SocketChannel ch, DataClientConnection clientConnection) {
protected GenericFutureListener<ChannelFuture>
getCloseHandler(SocketChannel ch, DataClientConnection clientConnection) {
return closeHandlerFactory.getHandler(clientConnection, super.getCloseHandler(ch, clientConnection));
}

Expand All @@ -75,27 +88,119 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}

@Override
protected Response handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
protected void handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
ResponseSender sender) throws RpcException {
throw new UnsupportedOperationException("DataClient is unidirectional by design.");
}

BufferAllocator getAllocator() {
return allocator;
return config.getAllocator();
}

@Override
protected void validateHandshake(BitServerHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != DataRpcConfig.RPC_VERSION) {
throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.",
handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
}

if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
final SaslClient saslClient;
try {
saslClient = config.getAuthFactory(handshake.getAuthenticationMechanismsList())
.createSaslClient(UserGroupInformation.getLoginUser(),
config.getSaslClientProperties(remoteEndpoint));
} catch (final IOException e) {
throw new RpcException(String.format("Failed to initiate authenticate to %s", remoteEndpoint.getAddress()), e);
}
if (saslClient == null) {
throw new RpcException("Unexpected failure. Could not initiate SASL exchange.");
}
connection.setSaslClient(saslClient);
} else {
if (config.getAuthMechanismToUse() != null) {
throw new RpcException(String.format("Drillbit (%s) does not require auth, but auth is enabled.",
remoteEndpoint.getAddress()));
}
}
}

@Override
protected void finalizeConnection(BitServerHandshake handshake, DataClientConnection connection) {
protected <M extends MessageLite> RpcCommand<M, DataClientConnection>
getInitialCommand(final RpcCommand<M, DataClientConnection> command) {
final RpcCommand<M, DataClientConnection> initialCommand = super.getInitialCommand(command);
if (config.getAuthMechanismToUse() == null) {
return initialCommand;
} else {
return new AuthenticationCommand<>(initialCommand);
}
}

public DataClientConnection getConnection() {
return this.connection;
private class AuthenticationCommand<M extends MessageLite> implements RpcCommand<M, DataClientConnection> {

private final RpcCommand<M, DataClientConnection> command;

AuthenticationCommand(RpcCommand<M, DataClientConnection> command) {
this.command = command;
}

@Override
public void connectionAvailable(DataClientConnection connection) {
command.connectionFailed(FailureType.AUTHENTICATION, new SaslException("Should not reach here."));
}

@Override
public void connectionSucceeded(final DataClientConnection connection) {
final UserGroupInformation loginUser;
try {
loginUser = UserGroupInformation.getLoginUser();
} catch (final IOException e) {
logger.debug("Unexpected failure trying to login.", e);
command.connectionFailed(FailureType.AUTHENTICATION, e);
return;
}

final SettableFuture<Void> future = SettableFuture.create();
new AuthenticationOutcomeListener<>(DataClient.this, connection, RpcType.SASL_MESSAGE,
loginUser,
new RpcOutcomeListener<Void>() {
@Override
public void failed(RpcException ex) {
logger.debug("Authentication failed.", ex);
future.setException(ex);
}

@Override
public void success(Void value, ByteBuf buffer) {
future.set(null);
}

@Override
public void interrupted(InterruptedException e) {
logger.debug("Authentication failed.", e);
future.setException(e);
}
}).initiate(config.getAuthMechanismToUse());

try {
logger.trace("Waiting until authentication completes..");
future.get();
command.connectionSucceeded(connection);
} catch (InterruptedException e) {
command.connectionFailed(FailureType.AUTHENTICATION, e);
// Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
// interruption and respond to it if it wants to.
Thread.currentThread().interrupt();
} catch (ExecutionException e) {
command.connectionFailed(FailureType.AUTHENTICATION, e);
}
}

@Override
public void connectionFailed(FailureType type, Throwable t) {
logger.debug("Authentication failed.", t);
command.connectionFailed(FailureType.AUTHENTICATION, t);
}
}

@Override
Expand Down
Expand Up @@ -24,21 +24,22 @@

import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.proto.BitData.RpcType;
import org.apache.drill.exec.rpc.RemoteConnection;
import org.apache.drill.exec.rpc.AbstractClientConnection;
import org.apache.drill.exec.rpc.RpcOutcomeListener;

import com.google.protobuf.MessageLite;
import org.slf4j.Logger;

public class DataClientConnection extends RemoteConnection{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataClientConnection.class);
// data connection on client-side (i.e. bit making request or sending data)
public class DataClientConnection extends AbstractClientConnection {
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataClientConnection.class);

private final DataClient client;
private final UUID id;

public DataClientConnection(SocketChannel channel, DataClient client) {
super(channel, "data client");
this.client = client;
// we use a local listener pool unless a global one is provided.
this.id = UUID.randomUUID();
}

Expand All @@ -47,10 +48,10 @@ public BufferAllocator getAllocator() {
return client.getAllocator();
}

public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(RpcOutcomeListener<RECEIVE> outcomeListener, RpcType rpcType,
SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
public <SEND extends MessageLite, RECEIVE extends MessageLite>
void send(RpcOutcomeListener<RECEIVE> outcomeListener, RpcType rpcType, SEND protobufBody,
Class<RECEIVE> clazz, ByteBuf... dataBodies) {
client.send(outcomeListener, this, rpcType, protobufBody, clazz, dataBodies);

}

@Override
Expand Down Expand Up @@ -83,5 +84,8 @@ public boolean equals(Object obj) {
return true;
}


@Override
protected Logger getLogger() {
return logger;
}
}
@@ -1,14 +1,14 @@
/**
/*
* 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
*
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
*
* <p>
* 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.
Expand All @@ -17,24 +17,30 @@
*/
package org.apache.drill.exec.rpc.data;

import io.netty.channel.socket.SocketChannel;

import org.apache.drill.exec.exception.DrillbitStartupException;
import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.rpc.RemoteConnection;
import org.apache.drill.exec.rpc.BitConnectionConfig;
import org.apache.drill.exec.server.BootStrapContext;

public class BitServerConnection extends RemoteConnection{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitServerConnection.class);
// config for bit to bit data connection
// package private
class DataConnectionConfig extends BitConnectionConfig {
// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataConnectionConfig.class);

private final BufferAllocator allocator;
private final DataServerRequestHandler handler;

public BitServerConnection(SocketChannel channel, BufferAllocator allocator) {
super(channel, "data server");
this.allocator = allocator;
DataConnectionConfig(BufferAllocator allocator, BootStrapContext context, DataServerRequestHandler handler)
throws DrillbitStartupException {
super(allocator, context);
this.handler = handler;
}

@Override
public BufferAllocator getAllocator() {
return allocator;
public String getName() {
return "data server";
}

DataServerRequestHandler getMessageHandler() {
return handler;
}
}
Expand Up @@ -37,40 +37,27 @@ public class DataConnectionCreator implements AutoCloseable {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataConnectionCreator.class);

private volatile DataServer server;
private final BootStrapContext context;
private final WorkEventBus workBus;
private final WorkerBee bee;
private final boolean allowPortHunting;
private ConcurrentMap<DrillbitEndpoint, DataConnectionManager> connectionManager = Maps.newConcurrentMap();
private final BufferAllocator dataAllocator;
private final ConcurrentMap<DrillbitEndpoint, DataConnectionManager> connectionManager = Maps.newConcurrentMap();

public DataConnectionCreator(
BootStrapContext context,
BufferAllocator allocator,
WorkEventBus workBus,
WorkerBee bee,
boolean allowPortHunting) {
super();
this.context = context;
this.workBus = workBus;
this.bee = bee;
this.allowPortHunting = allowPortHunting;
this.dataAllocator = allocator;
private final DataConnectionConfig config;

public DataConnectionCreator(BootStrapContext context, BufferAllocator allocator, WorkEventBus workBus,
WorkerBee bee) throws DrillbitStartupException {
config = new DataConnectionConfig(allocator, context, new DataServerRequestHandler(workBus, bee));
}

public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws DrillbitStartupException {
server = new DataServer(context, dataAllocator, workBus, bee);
public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint, boolean allowPortHunting) {
server = new DataServer(config);
int port = partialEndpoint.getControlPort() + 1;
if (context.getConfig().hasPath(ExecConstants.INITIAL_DATA_PORT)) {
port = context.getConfig().getInt(ExecConstants.INITIAL_DATA_PORT);
if (config.getBootstrapContext().getConfig().hasPath(ExecConstants.INITIAL_DATA_PORT)) {
port = config.getBootstrapContext().getConfig().getInt(ExecConstants.INITIAL_DATA_PORT);
}
port = server.bind(port, allowPortHunting);
DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setDataPort(port).build();
return completeEndpoint;
return partialEndpoint.toBuilder().setDataPort(port).build();
}

public DataTunnel getTunnel(DrillbitEndpoint endpoint) {
DataConnectionManager newManager = new DataConnectionManager(endpoint, context);
DataConnectionManager newManager = new DataConnectionManager(endpoint, config);
DataConnectionManager oldManager = connectionManager.putIfAbsent(endpoint, newManager);
if(oldManager != null){
newManager = oldManager;
Expand All @@ -80,7 +67,7 @@ public DataTunnel getTunnel(DrillbitEndpoint endpoint) {

@Override
public void close() throws Exception {
AutoCloseables.close(server, dataAllocator);
AutoCloseables.close(server, config.getAllocator());
}

}

0 comments on commit e2582a1

Please sign in to comment.