|
@@ -21,10 +21,13 @@ import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
|
import java.io.PrintStream;
|
|
import java.io.PrintStream;
|
|
|
import java.io.UnsupportedEncodingException;
|
|
import java.io.UnsupportedEncodingException;
|
|
|
|
|
+import java.lang.reflect.UndeclaredThrowableException;
|
|
|
import java.net.HttpURLConnection;
|
|
import java.net.HttpURLConnection;
|
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
|
import java.security.KeyManagementException;
|
|
import java.security.KeyManagementException;
|
|
|
import java.security.NoSuchAlgorithmException;
|
|
import java.security.NoSuchAlgorithmException;
|
|
|
|
|
+import java.security.PrivilegedActionException;
|
|
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
|
import java.security.SecureRandom;
|
|
import java.security.SecureRandom;
|
|
|
import java.text.SimpleDateFormat;
|
|
import java.text.SimpleDateFormat;
|
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
@@ -61,7 +64,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
|
|
|
import org.apache.hadoop.hive.ql.QueryPlan;
|
|
import org.apache.hadoop.hive.ql.QueryPlan;
|
|
|
import org.apache.hadoop.hive.ql.session.SessionState;
|
|
import org.apache.hadoop.hive.ql.session.SessionState;
|
|
|
import org.apache.hadoop.hive.serde.Constants;
|
|
import org.apache.hadoop.hive.serde.Constants;
|
|
|
-import org.apache.hadoop.security.UnixUserGroupInformation;
|
|
|
|
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.log4j.Logger;
|
|
import org.apache.log4j.Logger;
|
|
|
import org.apache.thrift.TException;
|
|
import org.apache.thrift.TException;
|
|
|
|
|
|
|
@@ -109,7 +112,7 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
* EXCEPTION is also valid.
|
|
* EXCEPTION is also valid.
|
|
|
*/
|
|
*/
|
|
|
private class RunningQueryState {
|
|
private class RunningQueryState {
|
|
|
- private int state = QueryState.CREATED;
|
|
|
|
|
|
|
+ private QueryState state = QueryState.CREATED;
|
|
|
// Thread local used by Hive quite a bit.
|
|
// Thread local used by Hive quite a bit.
|
|
|
private SessionState sessionState;
|
|
private SessionState sessionState;
|
|
|
private Throwable exception;
|
|
private Throwable exception;
|
|
@@ -183,7 +186,7 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
state = QueryState.COMPILED;
|
|
state = QueryState.COMPILED;
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
- private void assertState(int expected) {
|
|
|
|
|
|
|
+ private void assertState(QueryState expected) {
|
|
|
if (state != expected) {
|
|
if (state != expected) {
|
|
|
throw new IllegalStateException(String.format("Expected %s, but state is: %s",
|
|
throw new IllegalStateException(String.format("Expected %s, but state is: %s",
|
|
|
expected, state));
|
|
expected, state));
|
|
@@ -242,13 +245,6 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
if (query.hadoop_user == null) {
|
|
if (query.hadoop_user == null) {
|
|
|
throw new RuntimeException("User must be specified.");
|
|
throw new RuntimeException("User must be specified.");
|
|
|
}
|
|
}
|
|
|
- StringBuilder ugi = new StringBuilder();
|
|
|
|
|
- ugi.append(query.hadoop_user);
|
|
|
|
|
- for (String group : query.hadoop_groups) {
|
|
|
|
|
- ugi.append(",");
|
|
|
|
|
- ugi.append(group);
|
|
|
|
|
- }
|
|
|
|
|
- hiveConf.set(UnixUserGroupInformation.UGI_PROPERTY_NAME, ugi.toString());
|
|
|
|
|
|
|
|
|
|
// Update scratch dir (to have one per user)
|
|
// Update scratch dir (to have one per user)
|
|
|
File scratchDir = new File("/tmp/hive-beeswax-" + query.hadoop_user);
|
|
File scratchDir = new File("/tmp/hive-beeswax-" + query.hadoop_user);
|
|
@@ -448,10 +444,10 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
// Only one person can access a running query at a time.
|
|
// Only one person can access a running query at a time.
|
|
|
synchronized(this) {
|
|
synchronized(this) {
|
|
|
switch(state) {
|
|
switch(state) {
|
|
|
- case QueryState.RUNNING:
|
|
|
|
|
|
|
+ case RUNNING:
|
|
|
r.ready = false;
|
|
r.ready = false;
|
|
|
break;
|
|
break;
|
|
|
- case QueryState.FINISHED:
|
|
|
|
|
|
|
+ case FINISHED:
|
|
|
bringUp();
|
|
bringUp();
|
|
|
r.ready = true;
|
|
r.ready = true;
|
|
|
try {
|
|
try {
|
|
@@ -460,7 +456,7 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
throw new BeeswaxException(e.toString(), logContext.getName(), handle);
|
|
throw new BeeswaxException(e.toString(), logContext.getName(), handle);
|
|
|
}
|
|
}
|
|
|
break;
|
|
break;
|
|
|
- case QueryState.EXCEPTION:
|
|
|
|
|
|
|
+ case EXCEPTION:
|
|
|
if (exception instanceof BeeswaxException) {
|
|
if (exception instanceof BeeswaxException) {
|
|
|
throw (BeeswaxException) exception;
|
|
throw (BeeswaxException) exception;
|
|
|
} else {
|
|
} else {
|
|
@@ -607,6 +603,37 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
evicter.start();
|
|
evicter.start();
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
+
|
|
|
|
|
+ private <T> T doWithState(RunningQueryState state, PrivilegedExceptionAction<T> action)
|
|
|
|
|
+ throws BeeswaxException
|
|
|
|
|
+ {
|
|
|
|
|
+ try{
|
|
|
|
|
+ UserGroupInformation ugi;
|
|
|
|
|
+ if (UserGroupInformation.isSecurityEnabled())
|
|
|
|
|
+ ugi = UserGroupInformation.createProxyUser(state.query.hadoop_user, UserGroupInformation.getLoginUser());
|
|
|
|
|
+ else {
|
|
|
|
|
+ ugi = UserGroupInformation.createRemoteUser(state.query.hadoop_user);
|
|
|
|
|
+ }
|
|
|
|
|
+ return ugi.doAs(action);
|
|
|
|
|
+ } catch (UndeclaredThrowableException e) {
|
|
|
|
|
+ if (e.getUndeclaredThrowable() instanceof PrivilegedActionException) {
|
|
|
|
|
+ Throwable bwe = e.getUndeclaredThrowable().getCause();
|
|
|
|
|
+ if (bwe instanceof BeeswaxException) {
|
|
|
|
|
+ LOG.error("Caught BeeswaxException", (BeeswaxException) bwe);
|
|
|
|
|
+ throw (BeeswaxException) bwe;
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
|
|
+ LOG.error("Caught unexpected exception.", e);
|
|
|
|
|
+ throw new BeeswaxException(e.getMessage(), state.handle.log_context, state.handle);
|
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
|
+ LOG.error("Caught IOException", e);
|
|
|
|
|
+ throw new BeeswaxException(e.getMessage(), state.handle.log_context, state.handle);
|
|
|
|
|
+ } catch (InterruptedException e) {
|
|
|
|
|
+ LOG.error("Caught InterruptedException", e);
|
|
|
|
|
+ throw new BeeswaxException(e.getMessage(), state.handle.log_context, state.handle);
|
|
|
|
|
+ }
|
|
|
|
|
+ }
|
|
|
|
|
+
|
|
|
/**
|
|
/**
|
|
|
* Submit a query and return a handle (QueryHandle). The query runs asynchronously.
|
|
* Submit a query and return a handle (QueryHandle). The query runs asynchronously.
|
|
|
* Queries can be long-lasting, so we push the execution into a new state.
|
|
* Queries can be long-lasting, so we push the execution into a new state.
|
|
@@ -622,23 +649,31 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
|
|
|
|
|
// Make an administrative record
|
|
// Make an administrative record
|
|
|
final RunningQueryState state = new RunningQueryState(query, lc);
|
|
final RunningQueryState state = new RunningQueryState(query, lc);
|
|
|
- state.setQueryHandle(handle);
|
|
|
|
|
- runningQueries.put(handle.id, state);
|
|
|
|
|
- state.initialize();
|
|
|
|
|
- // All kinds of things can go wrong when we compile it. So catch all.
|
|
|
|
|
try {
|
|
try {
|
|
|
- state.compile();
|
|
|
|
|
- } catch (BeeswaxException perr) {
|
|
|
|
|
- state.saveException(perr);
|
|
|
|
|
- throw perr;
|
|
|
|
|
- } catch (Throwable t) {
|
|
|
|
|
- state.saveException(t);
|
|
|
|
|
- throw new BeeswaxException(t.toString(), handle.log_context, handle);
|
|
|
|
|
- }
|
|
|
|
|
- // Now spin off the query.
|
|
|
|
|
- state.submitTo(executor, lc);
|
|
|
|
|
-
|
|
|
|
|
- return handle;
|
|
|
|
|
|
|
+ return doWithState(state,
|
|
|
|
|
+ new PrivilegedExceptionAction<QueryHandle>() {
|
|
|
|
|
+ public QueryHandle run() throws Exception {
|
|
|
|
|
+ state.setQueryHandle(handle);
|
|
|
|
|
+ runningQueries.put(handle.id, state);
|
|
|
|
|
+ state.initialize();
|
|
|
|
|
+ // All kinds of things can go wrong when we compile it. So catch all.
|
|
|
|
|
+ try {
|
|
|
|
|
+ state.compile();
|
|
|
|
|
+ } catch (BeeswaxException perr) {
|
|
|
|
|
+ state.saveException(perr);
|
|
|
|
|
+ throw perr;
|
|
|
|
|
+ } catch (Throwable t) {
|
|
|
|
|
+ state.saveException(t);
|
|
|
|
|
+ throw new BeeswaxException(t.toString(), handle.log_context, handle);
|
|
|
|
|
+ }
|
|
|
|
|
+ // Now spin off the query.
|
|
|
|
|
+ state.submitTo(executor, lc);
|
|
|
|
|
+ return handle;
|
|
|
|
|
+ }
|
|
|
|
|
+ });
|
|
|
|
|
+ } catch (BeeswaxException e) {
|
|
|
|
|
+ throw e;
|
|
|
|
|
+ }
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -664,23 +699,32 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
* Get the query plan for a query.
|
|
* Get the query plan for a query.
|
|
|
*/
|
|
*/
|
|
|
@Override
|
|
@Override
|
|
|
- public QueryExplanation explain(Query query) throws BeeswaxException, TException {
|
|
|
|
|
- String contextName = UUID.randomUUID().toString();
|
|
|
|
|
|
|
+ public QueryExplanation explain(final Query query) throws BeeswaxException, TException {
|
|
|
|
|
+ final String contextName = UUID.randomUUID().toString();
|
|
|
LogContext lc = LogContext.registerCurrentThread(contextName);
|
|
LogContext lc = LogContext.registerCurrentThread(contextName);
|
|
|
- RunningQueryState state = new RunningQueryState(query, lc);
|
|
|
|
|
- state.initialize();
|
|
|
|
|
- // All kinds of things can go wrong when we compile it. So catch all.
|
|
|
|
|
- QueryExplanation exp;
|
|
|
|
|
|
|
+ final RunningQueryState state = new RunningQueryState(query, lc);
|
|
|
try {
|
|
try {
|
|
|
- exp = state.explain();
|
|
|
|
|
- } catch (BeeswaxException perr) {
|
|
|
|
|
- throw perr;
|
|
|
|
|
- } catch (Throwable t) {
|
|
|
|
|
- throw new BeeswaxException(t.toString(), contextName, null);
|
|
|
|
|
- }
|
|
|
|
|
- // On success, we remove the LogContext
|
|
|
|
|
- LogContext.destroyContext(contextName);
|
|
|
|
|
- return exp;
|
|
|
|
|
|
|
+ return doWithState(state,
|
|
|
|
|
+ new PrivilegedExceptionAction<QueryExplanation>() {
|
|
|
|
|
+ public QueryExplanation run() throws Exception {
|
|
|
|
|
+ state.initialize();
|
|
|
|
|
+ QueryExplanation exp;
|
|
|
|
|
+ // All kinds of things can go wrong when we compile it. So catch all.
|
|
|
|
|
+ try {
|
|
|
|
|
+ exp = state.explain();
|
|
|
|
|
+ } catch (BeeswaxException perr) {
|
|
|
|
|
+ throw perr;
|
|
|
|
|
+ } catch (Throwable t) {
|
|
|
|
|
+ throw new BeeswaxException(t.toString(), contextName, null);
|
|
|
|
|
+ }
|
|
|
|
|
+ // On success, we remove the LogContext
|
|
|
|
|
+ LogContext.destroyContext(contextName);
|
|
|
|
|
+ return exp;
|
|
|
|
|
+ }
|
|
|
|
|
+ });
|
|
|
|
|
+ } catch (BeeswaxException e) {
|
|
|
|
|
+ throw e;
|
|
|
|
|
+ }
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -691,17 +735,25 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
* @param fromBeginning If true, rewind to the first row. Otherwise fetch from last position.
|
|
* @param fromBeginning If true, rewind to the first row. Otherwise fetch from last position.
|
|
|
*/
|
|
*/
|
|
|
@Override
|
|
@Override
|
|
|
- public Results fetch(QueryHandle handle, boolean fromBeginning)
|
|
|
|
|
|
|
+ public Results fetch(final QueryHandle handle, final boolean fromBeginning)
|
|
|
throws QueryNotFoundException, BeeswaxException {
|
|
throws QueryNotFoundException, BeeswaxException {
|
|
|
LogContext.unregisterCurrentThread();
|
|
LogContext.unregisterCurrentThread();
|
|
|
validateHandle(handle);
|
|
validateHandle(handle);
|
|
|
LogContext.registerCurrentThread(handle.log_context);
|
|
LogContext.registerCurrentThread(handle.log_context);
|
|
|
- RunningQueryState state = runningQueries.get(handle.id);
|
|
|
|
|
- if (state == null) {
|
|
|
|
|
- throw new QueryNotFoundException();
|
|
|
|
|
|
|
+ final RunningQueryState state = runningQueries.get(handle.id);
|
|
|
|
|
+ try {
|
|
|
|
|
+ return doWithState(state,
|
|
|
|
|
+ new PrivilegedExceptionAction<Results>() {
|
|
|
|
|
+ public Results run() throws Exception {
|
|
|
|
|
+ if (state == null) {
|
|
|
|
|
+ throw new QueryNotFoundException();
|
|
|
|
|
+ }
|
|
|
|
|
+ return state.fetch(fromBeginning);
|
|
|
|
|
+ }
|
|
|
|
|
+ });
|
|
|
|
|
+ } catch (BeeswaxException e) {
|
|
|
|
|
+ throw e;
|
|
|
}
|
|
}
|
|
|
- Results res = state.fetch(fromBeginning);
|
|
|
|
|
- return res;
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -722,7 +774,7 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
* @param handle The handle from query()
|
|
* @param handle The handle from query()
|
|
|
*/
|
|
*/
|
|
|
@Override
|
|
@Override
|
|
|
- public int get_state(QueryHandle handle) throws QueryNotFoundException {
|
|
|
|
|
|
|
+ public QueryState get_state(final QueryHandle handle) throws QueryNotFoundException {
|
|
|
LogContext.unregisterCurrentThread();
|
|
LogContext.unregisterCurrentThread();
|
|
|
validateHandle(handle);
|
|
validateHandle(handle);
|
|
|
LogContext.registerCurrentThread(handle.log_context);
|
|
LogContext.registerCurrentThread(handle.log_context);
|
|
@@ -739,15 +791,25 @@ public class BeeswaxServiceImpl implements BeeswaxService.Iface {
|
|
|
* @param handle
|
|
* @param handle
|
|
|
*/
|
|
*/
|
|
|
@Override
|
|
@Override
|
|
|
- public ResultsMetadata get_results_metadata(QueryHandle handle) throws QueryNotFoundException {
|
|
|
|
|
|
|
+ public ResultsMetadata get_results_metadata(final QueryHandle handle) throws QueryNotFoundException {
|
|
|
LogContext.unregisterCurrentThread();
|
|
LogContext.unregisterCurrentThread();
|
|
|
validateHandle(handle);
|
|
validateHandle(handle);
|
|
|
LogContext.registerCurrentThread(handle.log_context);
|
|
LogContext.registerCurrentThread(handle.log_context);
|
|
|
- RunningQueryState state = runningQueries.get(handle.id);
|
|
|
|
|
- if (state == null) {
|
|
|
|
|
|
|
+ final RunningQueryState state = runningQueries.get(handle.id);
|
|
|
|
|
+ try {
|
|
|
|
|
+ return doWithState(state,
|
|
|
|
|
+ new PrivilegedExceptionAction<ResultsMetadata>() {
|
|
|
|
|
+ public ResultsMetadata run() throws Exception {
|
|
|
|
|
+ if (state == null) {
|
|
|
|
|
+ throw new QueryNotFoundException();
|
|
|
|
|
+ }
|
|
|
|
|
+ return state.getResultMetadata();
|
|
|
|
|
+ }
|
|
|
|
|
+ });
|
|
|
|
|
+ } catch (BeeswaxException e) {
|
|
|
|
|
+ LOG.error("Caught BeeswaxException.", e);
|
|
|
throw new QueryNotFoundException();
|
|
throw new QueryNotFoundException();
|
|
|
}
|
|
}
|
|
|
- return state.getResultMetadata();
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
/**
|