diff --git beeline/pom.xml beeline/pom.xml index 5503add..e7ba6a7 100644 --- beeline/pom.xml +++ beeline/pom.xml @@ -29,6 +29,7 @@ .. + 1.6.6 @@ -119,16 +120,23 @@ test - org.mockito - mockito-all - test - - postgresql postgresql 9.1-901.jdbc4 test + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-mockito + ${powermock.version} + test + diff --git beeline/src/java/org/apache/hive/beeline/Commands.java beeline/src/java/org/apache/hive/beeline/Commands.java index 748546d..fe4fd08 100644 --- beeline/src/java/org/apache/hive/beeline/Commands.java +++ beeline/src/java/org/apache/hive/beeline/Commands.java @@ -55,7 +55,6 @@ import org.apache.hadoop.hive.common.cli.ShellCmdExecutor; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.conf.HiveVariableSource; import org.apache.hadoop.hive.conf.SystemVariables; import org.apache.hadoop.hive.conf.VariableSubstitution; @@ -63,7 +62,10 @@ import org.apache.hive.jdbc.HiveStatement; import org.apache.hive.jdbc.Utils; import org.apache.hive.jdbc.Utils.JdbcConnectionParams; - +import org.apache.hadoop.hive.ql.log.InPlaceUpdate; +import org.apache.hadoop.hive.ql.log.ProgressMonitor; +import org.apache.hive.service.rpc.thrift.TJobExecutionStatus; +import org.apache.hive.service.rpc.thrift.TProgressUpdateResp; public class Commands { private final BeeLine beeLine; @@ -1242,43 +1244,140 @@ private void addCmdPart(List cmdList, StringBuffer command, String cmdpa command.setLength(0); } - private Runnable createLogRunnable(Statement statement) { + private Runnable createLogRunnable(final Statement statement) { if (statement instanceof HiveStatement) { - final HiveStatement hiveStatement = (HiveStatement) statement; - - Runnable runnable = new Runnable() { - @Override - public void run() { - while (hiveStatement.hasMoreLogs()) { - try { - // fetch the log periodically and output to beeline console - for (String log : hiveStatement.getQueryLog()) { - beeLine.info(log); - } - Thread.sleep(DEFAULT_QUERY_PROGRESS_INTERVAL); - } catch (SQLException e) { - beeLine.error(new SQLWarning(e)); - return; - } catch (InterruptedException e) { - beeLine.debug("Getting log thread is interrupted, since query is done!"); - showRemainingLogsIfAny(hiveStatement); - return; - } - } - } - }; - return runnable; + return new LogRunnable(this, (HiveStatement) statement, + DEFAULT_QUERY_PROGRESS_INTERVAL); } else { - beeLine.debug("The statement instance is not HiveStatement type: " + statement.getClass()); + beeLine.debug( + "The statement instance is not HiveStatement type: " + statement + .getClass()); return new Runnable() { - @Override - public void run() { + @Override public void run() { // do nothing. } }; } } + private void error(Throwable throwable) { + beeLine.error(throwable); + } + + private void showRemainingProgress(Statement statement, InPlaceUpdate inPlaceUpdate) { + if (statement instanceof HiveStatement) { + HiveStatement hiveStatement = (HiveStatement) statement; + try { + TProgressUpdateResp progressResponse = hiveStatement.getProgressResponse(); + boolean canRender = + progressResponse != null && TJobExecutionStatus.NOT_AVAILABLE != progressResponse + .getStatus(); + if (canRender) { + inPlaceUpdate.render(new ProgressMonitorWrapper(progressResponse)); + } + } catch (SQLException e) { + error(new SQLWarning(e)); + } + } + showRemainingLogsIfAny(statement); + } + + private void debug(String message) { + beeLine.debug(message); + } + + static class ProgressMonitorWrapper implements ProgressMonitor { + private TProgressUpdateResp response; + + ProgressMonitorWrapper(TProgressUpdateResp response) { + this.response = response; + } + + @Override public List headers() { + return response.getHeaderNames(); + } + + @Override public List> rows() { + return response.getRows(); + } + + @Override public String footerSummary() { + return response.getFooterSummary(); + } + + @Override public long startTime() { + return response.getStartTime(); + } + + @Override public String executionStatus() { + return response.getStatus().name(); + } + + @Override public double progressedPercentage() { + return response.getProgressedPercentage(); + } + } + + static class LogRunnable implements Runnable { + private final Commands commands; + private final HiveStatement hiveStatement; + private final long queryProgressInterval; + private final InPlaceUpdate inPlaceUpdate; + boolean progressBarUpdateCompleted = false; + + LogRunnable(Commands commands, HiveStatement hiveStatement, + long queryProgressInterval) { + this.hiveStatement = hiveStatement; + this.commands = commands; + this.queryProgressInterval = queryProgressInterval; + inPlaceUpdate = new InPlaceUpdate(commands.beeLine.getOutputStream()); + } + + private boolean receivedAllProgressUpdates(TJobExecutionStatus status) { + return TJobExecutionStatus.ERROR.equals(status) + || TJobExecutionStatus.SUBMITTED.equals(status) + || TJobExecutionStatus.KILLED.equals(status); + } + + private void updateQueryLog() throws SQLException { + for (String log : hiveStatement.getQueryLog()) { + commands.beeLine.info(log); + } + } + + @Override public void run() { + while (hiveStatement.hasMoreLogs()) { + try { + if (progressBarUpdateCompleted) { + updateQueryLog(); + } else { + TProgressUpdateResp progressResponse = hiveStatement.getProgressResponse(); + commands.debug("progress response sent: " + (progressResponse == null ? "null" : + progressResponse.toString())); + if (progressResponse == null || TJobExecutionStatus.NOT_AVAILABLE + .equals(progressResponse.getStatus())) { + progressBarUpdateCompleted = true; + } else { + inPlaceUpdate.render(new ProgressMonitorWrapper(progressResponse)); + if (receivedAllProgressUpdates(progressResponse.getStatus())) { + progressBarUpdateCompleted = true; + } + } + } + // fetch the log periodically and output to beeline console + // we are sleeping at the start so as to give the execute statement on server + // to setup the session state correctly before we make the call to get progress update + Thread.sleep(queryProgressInterval); + } catch (SQLException e) { + commands.error(new SQLWarning(e)); + } catch (InterruptedException e) { + commands.debug("Getting log thread is interrupted, since query is done!"); + commands.showRemainingProgress(hiveStatement, inPlaceUpdate); + } + } + } + } + private void showRemainingLogsIfAny(Statement statement) { if (statement instanceof HiveStatement) { HiveStatement hiveStatement = (HiveStatement) statement; diff --git beeline/src/test/org/apache/hive/beeline/TestLogRunnable.java beeline/src/test/org/apache/hive/beeline/TestLogRunnable.java new file mode 100644 index 0000000..6337c64 --- /dev/null +++ beeline/src/test/org/apache/hive/beeline/TestLogRunnable.java @@ -0,0 +1,86 @@ +package org.apache.hive.beeline; + +import org.apache.hadoop.hive.ql.log.InPlaceUpdate; +import org.apache.hadoop.hive.ql.log.ProgressMonitor; +import org.apache.hive.jdbc.HiveStatement; +import org.apache.hive.service.rpc.thrift.TJobExecutionStatus; +import org.apache.hive.service.rpc.thrift.TProgressUpdateResp; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.powermock.api.mockito.PowerMockito.verifyNoMoreInteractions; +import static org.powermock.api.mockito.PowerMockito.when; +import static org.powermock.api.mockito.PowerMockito.whenNew; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({InPlaceUpdate.class, Commands.LogRunnable.class}) +@PowerMockIgnore("javax.management.*") +public class TestLogRunnable { + + @Mock + private Commands commands; + @Mock + private HiveStatement hiveStatement; + @Mock + private InPlaceUpdate inplaceUpdate; + private Commands.LogRunnable logRunnable; + @Mock + private BeeLine beeline; + + @Before + public void setup() throws Exception { + Whitebox.setInternalState(commands, "beeLine", beeline); + whenNew(InPlaceUpdate.class).withAnyArguments().thenReturn(inplaceUpdate); + logRunnable = new Commands.LogRunnable(commands, hiveStatement, 0L); + } + + @Test + public void shouldNotQueryLogWhenInPlaceUpdateAvailable() throws Exception { + when(hiveStatement.hasMoreLogs()).thenReturn(true).thenReturn(false); + TProgressUpdateResp response = new TProgressUpdateResp( + Collections.emptyList(), new ArrayList>(), + 0.0D, + TJobExecutionStatus.INITING, + "", 0L); + when(hiveStatement.getProgressResponse()).thenReturn(response); + + logRunnable.run(); + + verify(hiveStatement, times(2)).hasMoreLogs(); + verify(hiveStatement).getProgressResponse(); + verifyNoMoreInteractions(hiveStatement); + verify(inplaceUpdate).render(any(ProgressMonitor.class)); + } + + @Test + public void shouldNotTryToRenderWhenJobStatusIsNotAvailable() throws Exception { + when(hiveStatement.hasMoreLogs()).thenReturn(true).thenReturn(false); + when(hiveStatement.getProgressResponse()).thenReturn(new TProgressUpdateResp( + Collections.emptyList(), Collections.>emptyList(), + 0.0D, + TJobExecutionStatus.NOT_AVAILABLE, + "", 0L)); + + logRunnable.run(); + + verify(hiveStatement, times(2)).hasMoreLogs(); + verify(hiveStatement).getProgressResponse(); + verifyNoMoreInteractions(hiveStatement); + verifyZeroInteractions(inplaceUpdate); + } +} \ No newline at end of file diff --git common/pom.xml common/pom.xml index e54ec98..2009e96 100644 --- common/pom.xml +++ common/pom.xml @@ -65,6 +65,11 @@ ${commons-lang3.version} + jline + jline + ${jline.version} + + org.eclipse.jetty.aggregate jetty-all ${jetty.version} diff --git common/src/java/org/apache/hadoop/hive/ql/log/InPlaceUpdate.java common/src/java/org/apache/hadoop/hive/ql/log/InPlaceUpdate.java new file mode 100644 index 0000000..4bb8f5b --- /dev/null +++ common/src/java/org/apache/hadoop/hive/ql/log/InPlaceUpdate.java @@ -0,0 +1,199 @@ +package org.apache.hadoop.hive.ql.log; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import jline.TerminalFactory; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.fusesource.jansi.Ansi; + +import javax.annotation.Nullable; +import java.io.PrintStream; +import java.io.StringWriter; +import java.text.DecimalFormat; +import java.util.List; + +import static org.fusesource.jansi.Ansi.ansi; +import static org.fusesource.jansi.internal.CLibrary.*; + +public class InPlaceUpdate { + + public static final int MIN_TERMINAL_WIDTH = 94; + + // keep this within 80 chars width. If more columns needs to be added then update min terminal + // width requirement and SEPARATOR width accordingly + private static final String HEADER_FORMAT = "%16s%10s %13s %5s %9s %7s %7s %6s %6s "; + private static final String VERTEX_FORMAT = "%-16s%10s %13s %5s %9s %7s %7s %6s %6s "; + private static final String FOOTER_FORMAT = "%-15s %-30s %-4s %-25s"; + + private static final int PROGRESS_BAR_CHARS = 30; + private static final String SEPARATOR = new String(new char[MIN_TERMINAL_WIDTH]).replace("\0", "-"); + + /* Pretty print the values */ + private final DecimalFormat secondsFormatter = new DecimalFormat("#0.00"); + private int lines = 0; + private PrintStream out; + + public InPlaceUpdate(PrintStream out) { + this.out = out; + } + + public InPlaceUpdate() { + this(System.out); + } + + public static void reprintLine(PrintStream out, String line) { + out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString()); + out.flush(); + } + + public static void rePositionCursor(PrintStream ps) { + ps.print(ansi().cursorUp(0).toString()); + ps.flush(); + } + + /** + * NOTE: Use this method only if isUnixTerminal is true. + * Erases the current line and prints the given line. + * + * @param line - line to print + */ + private void reprintLine(String line) { + reprintLine(out, line); + lines++; + } + + /** + * NOTE: Use this method only if isUnixTerminal is true. + * Erases the current line and prints the given line with the specified color. + * + * @param line - line to print + * @param color - color for the line + */ + private void reprintLineWithColorAsBold(String line, Ansi.Color color) { + out.print(ansi().eraseLine(Ansi.Erase.ALL).fg(color).bold().a(line).a('\n').boldOff().reset() + .toString()); + out.flush(); + lines++; + } + + /** + * NOTE: Use this method only if isUnixTerminal is true. + * Erases the current line and prints the given multiline. Make sure the specified line is not + * terminated by linebreak. + * + * @param line - line to print + */ + private void reprintMultiLine(String line) { + int numLines = line.split("\r\n|\r|\n").length; + out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString()); + out.flush(); + lines += numLines; + } + + /** + * NOTE: Use this method only if isUnixTerminal is true. + * Repositions the cursor back to line 0. + */ + private void repositionCursor() { + if (lines > 0) { + out.print(ansi().cursorUp(lines).toString()); + out.flush(); + lines = 0; + } + } + + + // [==================>>-----] + private String getInPlaceProgressBar(double percent) { + StringWriter bar = new StringWriter(); + bar.append("["); + int remainingChars = PROGRESS_BAR_CHARS - 4; + int completed = (int) (remainingChars * percent); + int pending = remainingChars - completed; + for (int i = 0; i < completed; i++) { + bar.append("="); + } + bar.append(">>"); + for (int i = 0; i < pending; i++) { + bar.append("-"); + } + bar.append("]"); + return bar.toString(); + } + + public void render(ProgressMonitor monitor) { + if (monitor == null) return; + // position the cursor to line 0 + repositionCursor(); + + // print header + // ------------------------------------------------------------------------------- + // VERTICES STATUS TOTAL COMPLETED RUNNING PENDING FAILED KILLED + // ------------------------------------------------------------------------------- + reprintLine(SEPARATOR); + reprintLineWithColorAsBold(String.format(HEADER_FORMAT, monitor.headers().toArray()), + Ansi.Color.CYAN); + reprintLine(SEPARATOR); + + + // Map 1 .......... container SUCCEEDED 7 7 0 0 0 0 + List printReady = Lists.transform(monitor.rows(), new Function, String>() { + @Nullable + @Override + public String apply(@Nullable List row) { + return String.format(VERTEX_FORMAT, row.toArray()); + } + }); + reprintMultiLine(StringUtils.join(printReady, "\n")); + + // ------------------------------------------------------------------------------- + // VERTICES: 03/04 [=================>>-----] 86% ELAPSED TIME: 1.71 s + // ------------------------------------------------------------------------------- + String progressStr = "" + (int) (monitor.progressedPercentage() * 100) + "%"; + float et = (float) (System.currentTimeMillis() - monitor.startTime()) / (float) 1000; + String elapsedTime = "ELAPSED TIME: " + secondsFormatter.format(et) + " s"; + String footer = String.format( + FOOTER_FORMAT, + monitor.footerSummary(), + getInPlaceProgressBar(monitor.progressedPercentage()), + progressStr, + elapsedTime); + + reprintLineWithColorAsBold(footer, Ansi.Color.RED); + reprintLine(SEPARATOR); + } + + + public static boolean canRenderInPlace(HiveConf conf) { + boolean inPlaceUpdates = HiveConf.getBoolVar(conf, HiveConf.ConfVars.TEZ_EXEC_INPLACE_PROGRESS); + + // we need at least 80 chars wide terminal to display in-place updates properly + return inPlaceUpdates && isUnixTerminal() && TerminalFactory.get().getWidth() >= MIN_TERMINAL_WIDTH; + } + + private static boolean isUnixTerminal() { + + String os = System.getProperty("os.name"); + if (os.startsWith("Windows")) { + // we do not support Windows, we will revisit this if we really need it for windows. + return false; + } + + // We must be on some unix variant.. + // check if standard out is a terminal + try { + // isatty system call will return 1 if the file descriptor is terminal else 0 + if (isatty(STDOUT_FILENO) == 0) { + return false; + } + if (isatty(STDERR_FILENO) == 0) { + return false; + } + } catch (NoClassDefFoundError | UnsatisfiedLinkError ignore) { + // These errors happen if the JNI lib is not available for your platform. + return false; + } + return true; + } +} diff --git common/src/java/org/apache/hadoop/hive/ql/log/ProgressMonitor.java common/src/java/org/apache/hadoop/hive/ql/log/ProgressMonitor.java new file mode 100644 index 0000000..cfb9291 --- /dev/null +++ common/src/java/org/apache/hadoop/hive/ql/log/ProgressMonitor.java @@ -0,0 +1,51 @@ +package org.apache.hadoop.hive.ql.log; + +import java.util.Collections; +import java.util.List; + +public interface ProgressMonitor { + + ProgressMonitor NULL = new ProgressMonitor() { + @Override + public List headers() { + return Collections.emptyList(); + } + + @Override + public List> rows() { + return Collections.emptyList(); + } + + @Override + public String footerSummary() { + return ""; + } + + @Override + public long startTime() { + return 0; + } + + @Override + public String executionStatus() { + return ""; + } + + @Override + public double progressedPercentage() { + return 0; + } + }; + + List headers(); + + List> rows(); + + String footerSummary(); + + long startTime(); + + String executionStatus(); + + double progressedPercentage(); +} diff --git itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java index 8b5b516..7144f1b 100644 --- itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java +++ itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java @@ -17,14 +17,14 @@ */ package org.apache.hive.service.cli.operation; -import java.util.HashMap; - import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hive.jdbc.miniHS2.MiniHS2; import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType; import org.junit.BeforeClass; +import java.util.HashMap; + /** * TestOperationLoggingAPIWithTez * Test the FetchResults of TFetchType.LOG in thrift level in Tez mode. @@ -50,7 +50,7 @@ public static void setUpBeforeClass() throws Exception { "", "", "", - "from=org.apache.hadoop.hive.ql.exec.tez.TezJobMonitor", + "from=org.apache.hadoop.hive.ql.exec.tez.monitoring.TezJobMonitor", "org.apache.tez.common.counters.DAGCounter", "NUM_SUCCEEDED_TASKS", "TOTAL_LAUNCHED_TASKS", diff --git jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java index a242501..40d4463 100644 --- jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java +++ jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java @@ -34,6 +34,8 @@ import org.apache.hive.service.rpc.thrift.TGetOperationStatusReq; import org.apache.hive.service.rpc.thrift.TGetOperationStatusResp; import org.apache.hive.service.rpc.thrift.TOperationHandle; +import org.apache.hive.service.rpc.thrift.TProgressUpdateReq; +import org.apache.hive.service.rpc.thrift.TProgressUpdateResp; import org.apache.hive.service.rpc.thrift.TSessionHandle; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -951,4 +953,17 @@ public String getYarnATSGuid() { } return null; } + + public TProgressUpdateResp getProgressResponse() throws SQLException { + if (stmtHandle == null) { + if (isQueryClosed || isExecuteStatementFailed) return null; + throw new SQLException("Method getProgressResponse() failed. Because the stmtHandle in " + + "HiveStatement is not yet set and the progress update is not yet available for querying."); + } + try { + return client.GetProgressUpdate(new TProgressUpdateReq(stmtHandle)); + } catch (TException e) { + throw new SQLException(e); + } + } } diff --git ql/pom.xml ql/pom.xml index 489c6f3..ce4139d 100644 --- ql/pom.xml +++ ql/pom.xml @@ -463,11 +463,6 @@ test - jline - jline - ${jline.version} - - org.apache.tez tez-api ${tez.version} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/InPlaceUpdates.java ql/src/java/org/apache/hadoop/hive/ql/exec/InPlaceUpdates.java deleted file mode 100644 index f59d8e2..0000000 --- ql/src/java/org/apache/hadoop/hive/ql/exec/InPlaceUpdates.java +++ /dev/null @@ -1,89 +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.hive.ql.exec; - -import static org.fusesource.jansi.Ansi.ansi; -import static org.fusesource.jansi.internal.CLibrary.STDERR_FILENO; -import static org.fusesource.jansi.internal.CLibrary.STDOUT_FILENO; -import static org.fusesource.jansi.internal.CLibrary.isatty; - -import java.io.PrintStream; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.fusesource.jansi.Ansi; - -import jline.TerminalFactory; - -public class InPlaceUpdates { - - public static final int MIN_TERMINAL_WIDTH = 94; - - static boolean isUnixTerminal() { - - String os = System.getProperty("os.name"); - if (os.startsWith("Windows")) { - // we do not support Windows, we will revisit this if we really need it for windows. - return false; - } - - // We must be on some unix variant.. - // check if standard out is a terminal - try { - // isatty system call will return 1 if the file descriptor is terminal else 0 - if (isatty(STDOUT_FILENO) == 0) { - return false; - } - if (isatty(STDERR_FILENO) == 0) { - return false; - } - } catch (NoClassDefFoundError ignore) { - // These errors happen if the JNI lib is not available for your platform. - return false; - } catch (UnsatisfiedLinkError ignore) { - // These errors happen if the JNI lib is not available for your platform. - return false; - } - return true; - } - - public static boolean inPlaceEligible(HiveConf conf) { - String engine = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE); - boolean inPlaceUpdates = false; - if (engine.equals("tez")) { - inPlaceUpdates = HiveConf.getBoolVar(conf, HiveConf.ConfVars.TEZ_EXEC_INPLACE_PROGRESS); - } - if (engine.equals("spark")) { - inPlaceUpdates = HiveConf.getBoolVar(conf, HiveConf.ConfVars.SPARK_EXEC_INPLACE_PROGRESS); - } - - // we need at least 80 chars wide terminal to display in-place updates properly - return inPlaceUpdates && !SessionState.getConsole().getIsSilent() && isUnixTerminal() - && TerminalFactory.get().getWidth() >= MIN_TERMINAL_WIDTH; - } - - public static void reprintLine(PrintStream out, String line) { - out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString()); - out.flush(); - } - - public static void rePositionCursor(PrintStream ps) { - ps.print(ansi().cursorUp(0).toString()); - ps.flush(); - } -} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java index 7be628e..247d589 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java @@ -38,7 +38,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.exec.tez.TezJobMonitor; import org.apache.hadoop.hive.ql.exec.vector.VectorFileSinkOperator; import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java index d5b9b5d..a6a45c7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hive.ql.exec.spark.status; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.InPlaceUpdates; +import org.apache.hadoop.hive.ql.log.InPlaceUpdate; import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.session.SessionState; import org.fusesource.jansi.Ansi; @@ -82,7 +82,7 @@ protected SparkJobMonitor(HiveConf hiveConf) { monitorTimeoutInterval = hiveConf.getTimeVar( HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT, TimeUnit.SECONDS); - inPlaceUpdate = InPlaceUpdates.inPlaceEligible(hiveConf); + inPlaceUpdate = InPlaceUpdate.canRenderInPlace(hiveConf) && !SessionState.getConsole().getIsSilent(); console = SessionState.getConsole(); out = SessionState.LogHelper.getInfoStream(); } @@ -270,7 +270,7 @@ private void repositionCursor() { } private void reprintLine(String line) { - InPlaceUpdates.reprintLine(out, line); + InPlaceUpdate.reprintLine(out, line); lines++; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobExecHelper.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobExecHelper.java index a3fc815..a544b93 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobExecHelper.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobExecHelper.java @@ -18,10 +18,11 @@ package org.apache.hadoop.hive.ql.exec.tez; -import java.lang.reflect.Method; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.Method; + /** * TezJobExecHelper is a utility to safely call Tez functionality from * common code paths. It will check if tez is available/installed before @@ -37,7 +38,7 @@ public static void killRunningJobs() { // we have tez installed ClassLoader classLoader = TezJobExecHelper.class.getClassLoader(); - Method method = classLoader.loadClass("org.apache.hadoop.hive.ql.exec.tez.TezJobMonitor") + Method method = classLoader.loadClass("org.apache.hadoop.hive.ql.exec.tez.monitoring.TezJobMonitor") .getMethod("killRunningJobs"); method.invoke(null, null); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java deleted file mode 100644 index bd935d4..0000000 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java +++ /dev/null @@ -1,1016 +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.hive.ql.exec.tez; - -import static org.apache.tez.dag.api.client.DAGStatus.State.RUNNING; -import static org.fusesource.jansi.Ansi.ansi; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.io.PrintStream; -import java.text.DecimalFormat; -import java.text.NumberFormat; -import java.util.EnumSet; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.llap.counters.LlapIOCounters; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.InPlaceUpdates; -import org.apache.hadoop.hive.ql.exec.MapOperator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; -import org.apache.hive.common.util.ShutdownHookManager; -import org.apache.tez.common.counters.FileSystemCounter; -import org.apache.tez.common.counters.TaskCounter; -import org.apache.tez.common.counters.TezCounter; -import org.apache.tez.common.counters.TezCounters; -import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.TezException; -import org.apache.tez.dag.api.Vertex; -import org.apache.tez.dag.api.client.DAGClient; -import org.apache.tez.dag.api.client.DAGStatus; -import org.apache.tez.dag.api.client.Progress; -import org.apache.tez.dag.api.client.StatusGetOpts; -import org.apache.tez.dag.api.client.VertexStatus; -import org.fusesource.jansi.Ansi; - -import com.google.common.base.Preconditions; - -/** - * TezJobMonitor keeps track of a tez job while it's being executed. It will - * print status to the console and retrieve final status of the job after - * completion. - */ -public class TezJobMonitor { - - private static final String CLASS_NAME = TezJobMonitor.class.getName(); - - private static final int COLUMN_1_WIDTH = 16; - private static final int SEPARATOR_WIDTH = InPlaceUpdates.MIN_TERMINAL_WIDTH; - private static final int FILE_HEADER_SEPARATOR_WIDTH = InPlaceUpdates.MIN_TERMINAL_WIDTH + 34; - private static final String SEPARATOR = new String(new char[SEPARATOR_WIDTH]).replace("\0", "-"); - private static final String FILE_HEADER_SEPARATOR = - new String(new char[FILE_HEADER_SEPARATOR_WIDTH]).replace("\0", "-"); - private static final String QUERY_EXEC_SUMMARY_HEADER = "Query Execution Summary"; - private static final String TASK_SUMMARY_HEADER = "Task Execution Summary"; - private static final String LLAP_IO_SUMMARY_HEADER = "LLAP IO Summary"; - private static final String FS_COUNTERS_SUMMARY_HEADER = "FileSystem Counters Summary"; - - // keep this within 80 chars width. If more columns needs to be added then update min terminal - // width requirement and SEPARATOR width accordingly - private static final String HEADER_FORMAT = "%16s%10s %13s %5s %9s %7s %7s %6s %6s "; - private static final String VERTEX_FORMAT = "%-16s%10s %13s %5s %9s %7s %7s %6s %6s "; - private static final String FOOTER_FORMAT = "%-15s %-30s %-4s %-25s"; - private static final String HEADER = String.format(HEADER_FORMAT, - "VERTICES", "MODE", "STATUS", "TOTAL", "COMPLETED", "RUNNING", "PENDING", "FAILED", "KILLED"); - - // method and dag summary format - private static final String SUMMARY_HEADER_FORMAT = "%10s %14s %13s %12s %14s %15s"; - private static final String SUMMARY_HEADER = String.format(SUMMARY_HEADER_FORMAT, - "VERTICES", "DURATION(ms)", "CPU_TIME(ms)", "GC_TIME(ms)", "INPUT_RECORDS", "OUTPUT_RECORDS"); - - // used when I/O redirection is used - private static final String FILE_HEADER_FORMAT = "%10s %12s %16s %13s %14s %13s %12s %14s %15s"; - private static final String FILE_HEADER = String.format(FILE_HEADER_FORMAT, - "VERTICES", "TOTAL_TASKS", "FAILED_ATTEMPTS", "KILLED_TASKS", "DURATION(ms)", - "CPU_TIME(ms)", "GC_TIME(ms)", "INPUT_RECORDS", "OUTPUT_RECORDS"); - - // LLAP counters - private static final String LLAP_SUMMARY_HEADER_FORMAT = "%10s %9s %9s %10s %9s %10s %11s %8s %9s"; - private static final String LLAP_SUMMARY_HEADER = String.format(LLAP_SUMMARY_HEADER_FORMAT, - "VERTICES", "ROWGROUPS", "META_HIT", "META_MISS", "DATA_HIT", "DATA_MISS", - "ALLOCATION", "USED", "TOTAL_IO"); - - // FileSystem counters - private static final String FS_COUNTERS_HEADER_FORMAT = "%10s %15s %13s %18s %18s %13s"; - - // Methods summary - private static final String OPERATION_SUMMARY = "%-35s %9s"; - private static final String OPERATION = "OPERATION"; - private static final String DURATION = "DURATION"; - - // in-place progress update related variables - private int lines; - private final PrintStream out; - - private transient LogHelper console; - private final PerfLogger perfLogger = SessionState.getPerfLogger(); - private final int checkInterval = 200; - private final int maxRetryInterval = 2500; - private final int printInterval = 3000; - private final int progressBarChars = 30; - private long lastPrintTime; - private Set completed; - - /* Pretty print the values */ - private final NumberFormat secondsFormat; - private final NumberFormat commaFormat; - private static final List shutdownList; - private final Map workMap; - - private StringBuffer diagnostics; - - static { - shutdownList = new LinkedList(); - ShutdownHookManager.addShutdownHook(new Runnable() { - @Override - public void run() { - TezJobMonitor.killRunningJobs(); - try { - TezSessionPoolManager.getInstance().closeNonDefaultSessions(false); - } catch (Exception e) { - // ignore - } - } - }); - } - - public static void initShutdownHook() { - Preconditions.checkNotNull(shutdownList, - "Shutdown hook was not properly initialized"); - } - - public TezJobMonitor(Map workMap) { - this.workMap = workMap; - console = SessionState.getConsole(); - secondsFormat = new DecimalFormat("#0.00"); - commaFormat = NumberFormat.getNumberInstance(Locale.US); - // all progress updates are written to info stream and log file. In-place updates can only be - // done to info stream (console) - out = console.getInfoStream(); - } - - /** - * NOTE: Use this method only if isUnixTerminal is true. - * Erases the current line and prints the given line. - * @param line - line to print - */ - public void reprintLine(String line) { - InPlaceUpdates.reprintLine(out, line); - lines++; - } - - /** - * NOTE: Use this method only if isUnixTerminal is true. - * Erases the current line and prints the given line with the specified color. - * @param line - line to print - * @param color - color for the line - */ - public void reprintLineWithColorAsBold(String line, Ansi.Color color) { - out.print(ansi().eraseLine(Ansi.Erase.ALL).fg(color).bold().a(line).a('\n').boldOff().reset() - .toString()); - out.flush(); - lines++; - } - - /** - * NOTE: Use this method only if isUnixTerminal is true. - * Erases the current line and prints the given multiline. Make sure the specified line is not - * terminated by linebreak. - * @param line - line to print - */ - public void reprintMultiLine(String line) { - int numLines = line.split("\r\n|\r|\n").length; - out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString()); - out.flush(); - lines += numLines; - } - - /** - * NOTE: Use this method only if isUnixTerminal is true. - * Repositions the cursor back to line 0. - */ - public void repositionCursor() { - if (lines > 0) { - out.print(ansi().cursorUp(lines).toString()); - out.flush(); - lines = 0; - } - } - - /** - * monitorExecution handles status printing, failures during execution and final status retrieval. - * - * @param dagClient client that was used to kick off the job - * @param conf configuration file for this operation - * @return int 0 - success, 1 - killed, 2 - failed - */ - public int monitorExecution(final DAGClient dagClient, HiveConf conf, - DAG dag, Context ctx) throws InterruptedException { - long monitorStartTime = System.currentTimeMillis(); - DAGStatus status = null; - completed = new HashSet(); - diagnostics = new StringBuffer(); - - boolean running = false; - boolean done = false; - boolean success = false; - int failedCounter = 0; - int rc = 0; - DAGStatus.State lastState = null; - String lastReport = null; - Set opts = new HashSet(); - long startTime = 0; - boolean isProfileEnabled = HiveConf.getBoolVar(conf, HiveConf.ConfVars.TEZ_EXEC_SUMMARY) || - Utilities.isPerfOrAboveLogging(conf); - boolean llapIoEnabled = HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_IO_ENABLED, false); - - boolean inPlaceEligible = InPlaceUpdates.inPlaceEligible(conf); - synchronized(shutdownList) { - shutdownList.add(dagClient); - } - console.printInfo("\n"); - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_DAG); - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_SUBMIT_TO_RUNNING); - Map progressMap = null; - while (true) { - - try { - if (ctx != null) { - ctx.checkHeartbeaterLockException(); - } - - status = dagClient.getDAGStatus(opts, checkInterval); - progressMap = status.getVertexProgress(); - DAGStatus.State state = status.getState(); - - if (state != lastState || state == RUNNING) { - lastState = state; - - switch (state) { - case SUBMITTED: - console.printInfo("Status: Submitted"); - break; - case INITING: - console.printInfo("Status: Initializing"); - startTime = System.currentTimeMillis(); - break; - case RUNNING: - if (!running) { - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_SUBMIT_TO_RUNNING); - console.printInfo("Status: Running (" + dagClient.getExecutionContext() + ")\n"); - startTime = System.currentTimeMillis(); - running = true; - } - - if (inPlaceEligible) { - printStatusInPlace(progressMap, startTime, false, dagClient); - // log the progress report to log file as well - lastReport = logStatus(progressMap, lastReport, console); - } else { - lastReport = printStatus(progressMap, lastReport, console); - } - break; - case SUCCEEDED: - if (!running) { - startTime = monitorStartTime; - } - if (inPlaceEligible) { - printStatusInPlace(progressMap, startTime, false, dagClient); - // log the progress report to log file as well - lastReport = logStatus(progressMap, lastReport, console); - } else { - lastReport = printStatus(progressMap, lastReport, console); - } - success = true; - running = false; - done = true; - break; - case KILLED: - if (!running) { - startTime = monitorStartTime; - } - if (inPlaceEligible) { - printStatusInPlace(progressMap, startTime, true, dagClient); - // log the progress report to log file as well - lastReport = logStatus(progressMap, lastReport, console); - } - console.printInfo("Status: Killed"); - running = false; - done = true; - rc = 1; - break; - case FAILED: - case ERROR: - if (!running) { - startTime = monitorStartTime; - } - if (inPlaceEligible) { - printStatusInPlace(progressMap, startTime, true, dagClient); - // log the progress report to log file as well - lastReport = logStatus(progressMap, lastReport, console); - } - console.printError("Status: Failed"); - running = false; - done = true; - rc = 2; - break; - } - } - } catch (Exception e) { - console.printInfo("Exception: " + e.getMessage()); - boolean isInterrupted = hasInterruptedException(e); - if (isInterrupted || (++failedCounter % maxRetryInterval / checkInterval == 0)) { - try { - console.printInfo("Killing DAG..."); - dagClient.tryKillDAG(); - } catch (IOException io) { - // best effort - } catch (TezException te) { - // best effort - } - e.printStackTrace(); - console.printError("Execution has failed."); - rc = 1; - done = true; - } else { - console.printInfo("Retrying..."); - } - } finally { - if (done) { - if (rc != 0 && status != null) { - for (String diag : status.getDiagnostics()) { - console.printError(diag); - diagnostics.append(diag); - } - } - synchronized(shutdownList) { - shutdownList.remove(dagClient); - } - break; - } - } - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_RUN_DAG); - - if (isProfileEnabled && success && progressMap != null) { - - double duration = (System.currentTimeMillis() - startTime) / 1000.0; - console.printInfo("Status: DAG finished successfully in " - + String.format("%.2f seconds", duration)); - console.printInfo(""); - - console.printInfo(QUERY_EXEC_SUMMARY_HEADER); - printQueryExecutionBreakDown(); - console.printInfo(SEPARATOR); - console.printInfo(""); - - console.printInfo(TASK_SUMMARY_HEADER); - printDagSummary(progressMap, console, dagClient, conf, dag, inPlaceEligible); - if (inPlaceEligible) { - console.printInfo(SEPARATOR); - } else { - console.printInfo(FILE_HEADER_SEPARATOR); - } - - if (llapIoEnabled) { - console.printInfo(""); - console.printInfo(LLAP_IO_SUMMARY_HEADER); - printLlapIOSummary(progressMap, console, dagClient); - console.printInfo(SEPARATOR); - console.printInfo(""); - - console.printInfo(FS_COUNTERS_SUMMARY_HEADER); - printFSCountersSummary(progressMap, console, dagClient); - } - - console.printInfo(""); - } - - return rc; - } - - private static boolean hasInterruptedException(Throwable e) { - // Hadoop IPC wraps InterruptedException. GRRR. - while (e != null) { - if (e instanceof InterruptedException || e instanceof InterruptedIOException) { - return true; - } - e = e.getCause(); - } - return false; - } - - /** - * killRunningJobs tries to terminate execution of all - * currently running tez queries. No guarantees, best effort only. - */ - public static void killRunningJobs() { - synchronized (shutdownList) { - for (DAGClient c : shutdownList) { - try { - System.err.println("Trying to shutdown DAG"); - c.tryKillDAG(); - } catch (Exception e) { - // ignore - } - } - } - } - - private static long getCounterValueByGroupName(TezCounters vertexCounters, - String groupNamePattern, - String counterName) { - TezCounter tezCounter = vertexCounters.getGroup(groupNamePattern).findCounter(counterName); - return (tezCounter == null) ? 0 : tezCounter.getValue(); - } - - private void printQueryExecutionBreakDown() { - - /* Build the method summary header */ - String execBreakdownHeader = String.format(OPERATION_SUMMARY, OPERATION, DURATION); - console.printInfo(SEPARATOR); - reprintLineWithColorAsBold(execBreakdownHeader, Ansi.Color.CYAN); - console.printInfo(SEPARATOR); - - // parse, analyze, optimize and compile - long compile = perfLogger.getEndTime(PerfLogger.COMPILE) - - perfLogger.getStartTime(PerfLogger.COMPILE); - console.printInfo(String.format(OPERATION_SUMMARY, "Compile Query", - secondsFormat.format(compile / 1000.0) + "s")); - - // prepare plan for submission (building DAG, adding resources, creating scratch dirs etc.) - long totalDAGPrep = perfLogger.getStartTime(PerfLogger.TEZ_SUBMIT_DAG) - - perfLogger.getEndTime(PerfLogger.COMPILE); - console.printInfo(String.format(OPERATION_SUMMARY, "Prepare Plan", - secondsFormat.format(totalDAGPrep / 1000.0) + "s")); - - // submit to accept dag (if session is closed, this will include re-opening of session time, - // localizing files for AM, submitting DAG) - long submitToAccept = perfLogger.getStartTime(PerfLogger.TEZ_RUN_DAG) - - perfLogger.getStartTime(PerfLogger.TEZ_SUBMIT_DAG); - console.printInfo(String.format(OPERATION_SUMMARY, "Submit Plan", - secondsFormat.format(submitToAccept / 1000.0) + "s")); - - // accept to start dag (schedule wait time, resource wait time etc.) - long acceptToStart = perfLogger.getDuration(PerfLogger.TEZ_SUBMIT_TO_RUNNING); - console.printInfo(String.format(OPERATION_SUMMARY, "Start DAG", - secondsFormat.format(acceptToStart / 1000.0) + "s")); - - // time to actually run the dag (actual dag runtime) - final long startToEnd; - if (acceptToStart == 0) { - startToEnd = perfLogger.getDuration(PerfLogger.TEZ_RUN_DAG); - } else { - startToEnd = perfLogger.getEndTime(PerfLogger.TEZ_RUN_DAG) - - perfLogger.getEndTime(PerfLogger.TEZ_SUBMIT_TO_RUNNING); - } - console.printInfo(String.format(OPERATION_SUMMARY, "Run DAG", - secondsFormat.format(startToEnd / 1000.0) + "s")); - - } - - private void printDagSummary(Map progressMap, LogHelper console, - DAGClient dagClient, HiveConf conf, DAG dag, final boolean inPlaceEligible) { - - /* Strings for headers and counters */ - String hiveCountersGroup = HiveConf.getVar(conf, HiveConf.ConfVars.HIVECOUNTERGROUP); - Set statusGetOpts = EnumSet.of(StatusGetOpts.GET_COUNTERS); - TezCounters hiveCounters = null; - try { - hiveCounters = dagClient.getDAGStatus(statusGetOpts).getDAGCounters(); - } catch (IOException e) { - // best attempt, shouldn't really kill DAG for this - } catch (TezException e) { - // best attempt, shouldn't really kill DAG for this - } - - /* If the counters are missing there is no point trying to print progress */ - if (hiveCounters == null) { - return; - } - - /* Print the per Vertex summary */ - if (inPlaceEligible) { - console.printInfo(SEPARATOR); - reprintLineWithColorAsBold(SUMMARY_HEADER, Ansi.Color.CYAN); - console.printInfo(SEPARATOR); - } else { - console.printInfo(FILE_HEADER_SEPARATOR); - reprintLineWithColorAsBold(FILE_HEADER, Ansi.Color.CYAN); - console.printInfo(FILE_HEADER_SEPARATOR); - } - SortedSet keys = new TreeSet(progressMap.keySet()); - Set statusOptions = new HashSet(1); - statusOptions.add(StatusGetOpts.GET_COUNTERS); - for (String vertexName : keys) { - Progress progress = progressMap.get(vertexName); - if (progress != null) { - final int totalTasks = progress.getTotalTaskCount(); - final int failedTaskAttempts = progress.getFailedTaskAttemptCount(); - final int killedTaskAttempts = progress.getKilledTaskAttemptCount(); - final double duration = perfLogger.getDuration(PerfLogger.TEZ_RUN_VERTEX + vertexName); - VertexStatus vertexStatus = null; - try { - vertexStatus = dagClient.getVertexStatus(vertexName, statusOptions); - } catch (IOException e) { - // best attempt, shouldn't really kill DAG for this - } catch (TezException e) { - // best attempt, shouldn't really kill DAG for this - } - - if (vertexStatus == null) { - continue; - } - - Vertex currentVertex = dag.getVertex(vertexName); - List inputVerticesList = currentVertex.getInputVertices(); - long hiveInputRecordsFromOtherVertices = 0; - if (inputVerticesList.size() > 0) { - - for (Vertex inputVertex : inputVerticesList) { - String inputVertexName = inputVertex.getName(); - hiveInputRecordsFromOtherVertices += getCounterValueByGroupName(hiveCounters, - hiveCountersGroup, String.format("%s_", - ReduceSinkOperator.Counter.RECORDS_OUT_INTERMEDIATE.toString()) + - inputVertexName.replace(" ", "_")); - - hiveInputRecordsFromOtherVertices += getCounterValueByGroupName(hiveCounters, - hiveCountersGroup, String.format("%s_", - FileSinkOperator.Counter.RECORDS_OUT.toString()) + - inputVertexName.replace(" ", "_")); - } - } - - /* - * Get the CPU & GC - * - * counters org.apache.tez.common.counters.TaskCounter - * GC_TIME_MILLIS=37712 - * CPU_MILLISECONDS=2774230 - */ - final TezCounters vertexCounters = vertexStatus.getVertexCounters(); - final double cpuTimeMillis = getCounterValueByGroupName(vertexCounters, - TaskCounter.class.getName(), - TaskCounter.CPU_MILLISECONDS.name()); - - final double gcTimeMillis = getCounterValueByGroupName(vertexCounters, - TaskCounter.class.getName(), - TaskCounter.GC_TIME_MILLIS.name()); - - /* - * Get the HIVE counters - * - * HIVE - * CREATED_FILES=1 - * DESERIALIZE_ERRORS=0 - * RECORDS_IN_Map_1=550076554 - * RECORDS_OUT_INTERMEDIATE_Map_1=854987 - * RECORDS_OUT_Reducer_2=1 - */ - - final long hiveInputRecords = - getCounterValueByGroupName( - hiveCounters, - hiveCountersGroup, - String.format("%s_", MapOperator.Counter.RECORDS_IN.toString()) - + vertexName.replace(" ", "_")) - + hiveInputRecordsFromOtherVertices; - final long hiveOutputIntermediateRecords = - getCounterValueByGroupName( - hiveCounters, - hiveCountersGroup, - String.format("%s_", ReduceSinkOperator.Counter.RECORDS_OUT_INTERMEDIATE.toString()) - + vertexName.replace(" ", "_")); - final long hiveOutputRecords = - getCounterValueByGroupName( - hiveCounters, - hiveCountersGroup, - String.format("%s_", FileSinkOperator.Counter.RECORDS_OUT.toString()) - + vertexName.replace(" ", "_")) - + hiveOutputIntermediateRecords; - - final String vertexExecutionStats; - if (inPlaceEligible) { - vertexExecutionStats = String.format(SUMMARY_HEADER_FORMAT, - vertexName, - secondsFormat.format((duration)), - commaFormat.format(cpuTimeMillis), - commaFormat.format(gcTimeMillis), - commaFormat.format(hiveInputRecords), - commaFormat.format(hiveOutputRecords)); - } else { - vertexExecutionStats = String.format(FILE_HEADER_FORMAT, - vertexName, - totalTasks, - failedTaskAttempts, - killedTaskAttempts, - secondsFormat.format((duration)), - commaFormat.format(cpuTimeMillis), - commaFormat.format(gcTimeMillis), - commaFormat.format(hiveInputRecords), - commaFormat.format(hiveOutputRecords)); - } - console.printInfo(vertexExecutionStats); - } - } - } - - private void printLlapIOSummary(Map progressMap, LogHelper console, - DAGClient dagClient) { - SortedSet keys = new TreeSet<>(progressMap.keySet()); - Set statusOptions = new HashSet<>(1); - statusOptions.add(StatusGetOpts.GET_COUNTERS); - boolean first = false; - String counterGroup = LlapIOCounters.class.getName(); - for (String vertexName : keys) { - // Reducers do not benefit from LLAP IO so no point in printing - if (vertexName.startsWith("Reducer")) { - continue; - } - TezCounters vertexCounters = null; - try { - vertexCounters = dagClient.getVertexStatus(vertexName, statusOptions) - .getVertexCounters(); - } catch (IOException e) { - // best attempt, shouldn't really kill DAG for this - } catch (TezException e) { - // best attempt, shouldn't really kill DAG for this - } - if (vertexCounters != null) { - final long selectedRowgroups = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.SELECTED_ROWGROUPS.name()); - final long metadataCacheHit = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.METADATA_CACHE_HIT.name()); - final long metadataCacheMiss = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.METADATA_CACHE_MISS.name()); - final long cacheHitBytes = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.CACHE_HIT_BYTES.name()); - final long cacheMissBytes = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.CACHE_MISS_BYTES.name()); - final long allocatedBytes = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.ALLOCATED_BYTES.name()); - final long allocatedUsedBytes = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.ALLOCATED_USED_BYTES.name()); - final long totalIoTime = getCounterValueByGroupName(vertexCounters, - counterGroup, LlapIOCounters.TOTAL_IO_TIME_NS.name()); - - if (!first) { - console.printInfo(SEPARATOR); - reprintLineWithColorAsBold(LLAP_SUMMARY_HEADER, Ansi.Color.CYAN); - console.printInfo(SEPARATOR); - first = true; - } - - String queryFragmentStats = String.format(LLAP_SUMMARY_HEADER_FORMAT, - vertexName, - selectedRowgroups, - metadataCacheHit, - metadataCacheMiss, - Utilities.humanReadableByteCount(cacheHitBytes), - Utilities.humanReadableByteCount(cacheMissBytes), - Utilities.humanReadableByteCount(allocatedBytes), - Utilities.humanReadableByteCount(allocatedUsedBytes), - secondsFormat.format(totalIoTime / 1000_000_000.0) + "s"); - console.printInfo(queryFragmentStats); - } - } - } - - private void printFSCountersSummary(Map progressMap, LogHelper console, - DAGClient dagClient) { - SortedSet keys = new TreeSet<>(progressMap.keySet()); - Set statusOptions = new HashSet<>(1); - statusOptions.add(StatusGetOpts.GET_COUNTERS); - // Assuming FileSystem.getAllStatistics() returns all schemes that are accessed on task side - // as well. If not, we need a way to get all the schemes that are accessed by the tez task/llap. - for (FileSystem.Statistics statistics : FileSystem.getAllStatistics()) { - final String scheme = statistics.getScheme().toUpperCase(); - final String fsCountersHeader = String.format(FS_COUNTERS_HEADER_FORMAT, - "VERTICES", "BYTES_READ", "READ_OPS", "LARGE_READ_OPS", "BYTES_WRITTEN", "WRITE_OPS"); - - console.printInfo(""); - reprintLineWithColorAsBold("Scheme: " + scheme, Ansi.Color.RED); - console.printInfo(SEPARATOR); - reprintLineWithColorAsBold(fsCountersHeader, Ansi.Color.CYAN); - console.printInfo(SEPARATOR); - - for (String vertexName : keys) { - TezCounters vertexCounters = null; - try { - vertexCounters = dagClient.getVertexStatus(vertexName, statusOptions) - .getVertexCounters(); - } catch (IOException e) { - // best attempt, shouldn't really kill DAG for this - } catch (TezException e) { - // best attempt, shouldn't really kill DAG for this - } - if (vertexCounters != null) { - final String counterGroup = FileSystemCounter.class.getName(); - final long bytesRead = getCounterValueByGroupName(vertexCounters, - counterGroup, scheme + "_" + FileSystemCounter.BYTES_READ.name()); - final long bytesWritten = getCounterValueByGroupName(vertexCounters, - counterGroup, scheme + "_" + FileSystemCounter.BYTES_WRITTEN.name()); - final long readOps = getCounterValueByGroupName(vertexCounters, - counterGroup, scheme + "_" + FileSystemCounter.READ_OPS.name()); - final long largeReadOps = getCounterValueByGroupName(vertexCounters, - counterGroup, scheme + "_" + FileSystemCounter.LARGE_READ_OPS.name()); - final long writeOps = getCounterValueByGroupName(vertexCounters, - counterGroup, scheme + "_" + FileSystemCounter.WRITE_OPS.name()); - - String fsCountersSummary = String.format(FS_COUNTERS_HEADER_FORMAT, - vertexName, - Utilities.humanReadableByteCount(bytesRead), - readOps, - largeReadOps, - Utilities.humanReadableByteCount(bytesWritten), - writeOps); - console.printInfo(fsCountersSummary); - } - } - - console.printInfo(SEPARATOR); - } - } - - private void printStatusInPlace(Map progressMap, long startTime, - boolean vextexStatusFromAM, DAGClient dagClient) { - StringBuilder reportBuffer = new StringBuilder(); - int sumComplete = 0; - int sumTotal = 0; - - // position the cursor to line 0 - repositionCursor(); - - // print header - // ------------------------------------------------------------------------------- - // VERTICES STATUS TOTAL COMPLETED RUNNING PENDING FAILED KILLED - // ------------------------------------------------------------------------------- - reprintLine(SEPARATOR); - reprintLineWithColorAsBold(HEADER, Ansi.Color.CYAN); - reprintLine(SEPARATOR); - - SortedSet keys = new TreeSet(progressMap.keySet()); - int idx = 0; - int maxKeys = keys.size(); - for (String s : keys) { - idx++; - Progress progress = progressMap.get(s); - final int complete = progress.getSucceededTaskCount(); - final int total = progress.getTotalTaskCount(); - final int running = progress.getRunningTaskCount(); - final int failed = progress.getFailedTaskAttemptCount(); - final int pending = progress.getTotalTaskCount() - progress.getSucceededTaskCount() - - progress.getRunningTaskCount(); - final int killed = progress.getKilledTaskAttemptCount(); - - // To get vertex status we can use DAGClient.getVertexStatus(), but it will be expensive to - // get status from AM for every refresh of the UI. Lets infer the state from task counts. - // Only if DAG is FAILED or KILLED the vertex status is fetched from AM. - VertexStatus.State vertexState = VertexStatus.State.INITIALIZING; - - // INITED state - if (total > 0) { - vertexState = VertexStatus.State.INITED; - sumComplete += complete; - sumTotal += total; - } - - // RUNNING state - if (complete < total && (complete > 0 || running > 0 || failed > 0)) { - vertexState = VertexStatus.State.RUNNING; - if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); - } - } - - // SUCCEEDED state - if (complete == total) { - vertexState = VertexStatus.State.SUCCEEDED; - if (!completed.contains(s)) { - completed.add(s); - - /* We may have missed the start of the vertex - * due to the 3 seconds interval - */ - if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); - } - } - - // DAG might have been killed, lets try to get vertex state from AM before dying - // KILLED or FAILED state - if (vextexStatusFromAM) { - VertexStatus vertexStatus = null; - try { - vertexStatus = dagClient.getVertexStatus(s, null); - } catch (IOException e) { - // best attempt, shouldn't really kill DAG for this - } catch (TezException e) { - // best attempt, shouldn't really kill DAG for this - } - if (vertexStatus != null) { - vertexState = vertexStatus.getState(); - } - } - - // Map 1 .......... container SUCCEEDED 7 7 0 0 0 0 - String nameWithProgress = getNameWithProgress(s, complete, total); - String mode = getMode(s, workMap); - String vertexStr = String.format(VERTEX_FORMAT, - nameWithProgress, - mode, - vertexState.toString(), - total, - complete, - running, - pending, - failed, - killed); - reportBuffer.append(vertexStr); - if (idx != maxKeys) { - reportBuffer.append("\n"); - } - } - - reprintMultiLine(reportBuffer.toString()); - - // ------------------------------------------------------------------------------- - // VERTICES: 03/04 [=================>>-----] 86% ELAPSED TIME: 1.71 s - // ------------------------------------------------------------------------------- - reprintLine(SEPARATOR); - final float progress = (sumTotal == 0) ? 0.0f : (float) sumComplete / (float) sumTotal; - String footer = getFooter(keys.size(), completed.size(), progress, startTime); - reprintLineWithColorAsBold(footer, Ansi.Color.RED); - reprintLine(SEPARATOR); - } - - private String getMode(String name, Map workMap) { - String mode = "container"; - BaseWork work = workMap.get(name); - if (work != null) { - // uber > llap > container - if (work.getUberMode()) { - mode = "uber"; - } else if (work.getLlapMode()) { - mode = "llap"; - } else { - mode = "container"; - } - } - return mode; - } - - // Map 1 .......... - private String getNameWithProgress(String s, int complete, int total) { - String result = ""; - if (s != null) { - float percent = total == 0 ? 0.0f : (float) complete / (float) total; - // lets use the remaining space in column 1 as progress bar - int spaceRemaining = COLUMN_1_WIDTH - s.length() - 1; - String trimmedVName = s; - - // if the vertex name is longer than column 1 width, trim it down - // "Tez Merge File Work" will become "Tez Merge File.." - if (s.length() > COLUMN_1_WIDTH) { - trimmedVName = s.substring(0, COLUMN_1_WIDTH - 1); - trimmedVName = trimmedVName + ".."; - } - - result = trimmedVName + " "; - int toFill = (int) (spaceRemaining * percent); - for (int i = 0; i < toFill; i++) { - result += "."; - } - } - return result; - } - - // VERTICES: 03/04 [==================>>-----] 86% ELAPSED TIME: 1.71 s - private String getFooter(int keySize, int completedSize, float progress, long startTime) { - String verticesSummary = String.format("VERTICES: %02d/%02d", completedSize, keySize); - String progressBar = getInPlaceProgressBar(progress); - final int progressPercent = (int) (progress * 100); - String progressStr = "" + progressPercent + "%"; - float et = (float) (System.currentTimeMillis() - startTime) / (float) 1000; - String elapsedTime = "ELAPSED TIME: " + secondsFormat.format(et) + " s"; - String footer = String.format(FOOTER_FORMAT, - verticesSummary, progressBar, progressStr, elapsedTime); - return footer; - } - - // [==================>>-----] - private String getInPlaceProgressBar(float percent) { - StringBuilder bar = new StringBuilder("["); - int remainingChars = progressBarChars - 4; - int completed = (int) (remainingChars * percent); - int pending = remainingChars - completed; - for (int i = 0; i < completed; i++) { - bar.append("="); - } - bar.append(">>"); - for (int i = 0; i < pending; i++) { - bar.append("-"); - } - bar.append("]"); - return bar.toString(); - } - - private String printStatus(Map progressMap, String lastReport, LogHelper console) { - String report = getReport(progressMap); - if (!report.equals(lastReport) || System.currentTimeMillis() >= lastPrintTime + printInterval) { - console.printInfo(report); - lastPrintTime = System.currentTimeMillis(); - } - return report; - } - - private String logStatus(Map progressMap, String lastReport, LogHelper console) { - String report = getReport(progressMap); - if (!report.equals(lastReport) || System.currentTimeMillis() >= lastPrintTime + printInterval) { - console.logInfo(report); - lastPrintTime = System.currentTimeMillis(); - } - return report; - } - - private String getReport(Map progressMap) { - StringBuilder reportBuffer = new StringBuilder(); - - SortedSet keys = new TreeSet(progressMap.keySet()); - for (String s: keys) { - Progress progress = progressMap.get(s); - final int complete = progress.getSucceededTaskCount(); - final int total = progress.getTotalTaskCount(); - final int running = progress.getRunningTaskCount(); - final int failed = progress.getFailedTaskAttemptCount(); - if (total <= 0) { - reportBuffer.append(String.format("%s: -/-\t", s)); - } else { - if (complete == total && !completed.contains(s)) { - completed.add(s); - - /* - * We may have missed the start of the vertex due to the 3 seconds interval - */ - if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); - } - if(complete < total && (complete > 0 || running > 0 || failed > 0)) { - - if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); - } - - /* vertex is started, but not complete */ - if (failed > 0) { - reportBuffer.append(String.format("%s: %d(+%d,-%d)/%d\t", s, complete, running, failed, total)); - } else { - reportBuffer.append(String.format("%s: %d(+%d)/%d\t", s, complete, running, total)); - } - } else { - /* vertex is waiting for input/slots or complete */ - if (failed > 0) { - /* tasks finished but some failed */ - reportBuffer.append(String.format("%s: %d(-%d)/%d\t", s, complete, failed, total)); - } else { - reportBuffer.append(String.format("%s: %d/%d\t", s, complete, total)); - } - } - } - } - - return reportBuffer.toString(); - } - - public String getDiagnostics() { - return diagnostics.toString(); - } -} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java index e8a1757..9580434 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java @@ -17,9 +17,7 @@ */ package org.apache.hadoop.hive.ql.exec.tez; - import java.util.Collection; - import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -40,9 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; - import javax.security.auth.login.LoginException; - import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.io.FilenameUtils; import org.apache.hadoop.conf.Configuration; @@ -83,6 +79,7 @@ import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.ql.exec.tez.monitoring.TezJobMonitor; /** * Holds session state related to Tez @@ -666,7 +663,7 @@ public HiveConf getConf() { } public List getLocalizedResources() { - return new ArrayList(localizedResources); + return new ArrayList<>(localizedResources); } public String getUser() { @@ -693,4 +690,5 @@ public void markInUse() { } } while (!ownerThread.compareAndSet(null, newName)); } + } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java index 0efca68..d923c3f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java @@ -79,6 +79,7 @@ import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatus; import org.json.JSONObject; +import org.apache.hadoop.hive.ql.exec.tez.monitoring.TezJobMonitor; /** * @@ -178,8 +179,9 @@ public int execute(DriverContext driverContext) { additionalLr, inputOutputJars, inputOutputLocalResources); // finally monitor will print progress until the job is done - TezJobMonitor monitor = new TezJobMonitor(work.getWorkMap()); - rc = monitor.monitorExecution(dagClient, conf, dag, ctx); + TezJobMonitor monitor = new TezJobMonitor(work.getWorkMap(),dagClient, conf, dag, ctx); + rc = monitor.monitorExecution(); + if (rc != 0) { this.setException(new HiveException(monitor.getDiagnostics())); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/Constants.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/Constants.java new file mode 100644 index 0000000..4c7953d --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/Constants.java @@ -0,0 +1,7 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.hive.ql.log.InPlaceUpdate; + +public interface Constants { + String SEPARATOR = new String(new char[InPlaceUpdate.MIN_TERMINAL_WIDTH]).replace("\0", "-"); +} \ No newline at end of file diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java new file mode 100644 index 0000000..9d419fb --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java @@ -0,0 +1,197 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.MapOperator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.log.InPlaceUpdate; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.tez.common.counters.TaskCounter; +import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.common.counters.TezCounters; +import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.Vertex; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.Progress; +import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.dag.api.client.VertexStatus; + +import java.io.IOException; +import java.text.DecimalFormat; +import java.text.NumberFormat; +import java.util.*; + + +class DAGSummary implements PrintSummary { + + private static final int FILE_HEADER_SEPARATOR_WIDTH = InPlaceUpdate.MIN_TERMINAL_WIDTH + 34; + private static final String FILE_HEADER_SEPARATOR = new String(new char[FILE_HEADER_SEPARATOR_WIDTH]).replace("\0", "-"); + + private static final String FORMATTING_PATTERN = "%10s %12s %16s %13s %14s %13s %12s %14s %15s"; + private static final String FILE_HEADER = String.format( + FORMATTING_PATTERN, + "VERTICES", + "TOTAL_TASKS", + "FAILED_ATTEMPTS", + "KILLED_TASKS", + "DURATION(ms)", + "CPU_TIME(ms)", + "GC_TIME(ms)", + "INPUT_RECORDS", + "OUTPUT_RECORDS" + ); + + private final DecimalFormat secondsFormatter = new DecimalFormat("#0.00"); + private final NumberFormat commaFormatter = NumberFormat.getNumberInstance(Locale.US); + + private final String hiveCountersGroup; + private final TezCounters hiveCounters; + + private Map progressMap; + private DAGClient dagClient; + private DAG dag; + private PerfLogger perfLogger; + + DAGSummary(Map progressMap, HiveConf hiveConf, DAGClient dagClient, + DAG dag, PerfLogger perfLogger) { + this.progressMap = progressMap; + this.dagClient = dagClient; + this.dag = dag; + this.perfLogger = perfLogger; + this.hiveCountersGroup = HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVECOUNTERGROUP); + this.hiveCounters = hiveCounters(dagClient); + } + + private long hiveInputRecordsFromOtherVertices(String vertexName) { + List inputVerticesList = dag.getVertex(vertexName).getInputVertices(); + long result = 0; + for (Vertex inputVertex : inputVerticesList) { + String intermediateRecordsCounterName = formattedName( + ReduceSinkOperator.Counter.RECORDS_OUT_INTERMEDIATE.toString(), + inputVertex.getName() + ); + String recordsOutCounterName = formattedName(FileSinkOperator.Counter.RECORDS_OUT.toString(), + inputVertex.getName()); + result += ( + hiveCounterValue(intermediateRecordsCounterName) + + hiveCounterValue(recordsOutCounterName) + ); + } + return result; + } + + private String formattedName(String counterName, String vertexName) { + return String.format("%s_", counterName) + vertexName.replace(" ", "_"); + } + + private long getCounterValueByGroupName(TezCounters counters, String pattern, String counterName) { + TezCounter tezCounter = counters.getGroup(pattern).findCounter(counterName); + return (tezCounter == null) ? 0 : tezCounter.getValue(); + } + + private long hiveCounterValue(String counterName) { + return getCounterValueByGroupName(hiveCounters, hiveCountersGroup, counterName); + } + + private TezCounters hiveCounters(DAGClient dagClient) { + try { + return dagClient.getDAGStatus(EnumSet.of(StatusGetOpts.GET_COUNTERS)).getDAGCounters(); + } catch (IOException | TezException e) { + // best attempt, shouldn't really kill DAG for this + } + return null; + } + + @Override + public void print(SessionState.LogHelper console) { + console.printInfo("Task Execution Summary"); + + /* If the counters are missing there is no point trying to print progress */ + if (hiveCounters == null) { + return; + } + + /* Print the per Vertex summary */ + printHeader(console); + SortedSet keys = new TreeSet<>(progressMap.keySet()); + Set statusOptions = new HashSet<>(1); + statusOptions.add(StatusGetOpts.GET_COUNTERS); + for (String vertexName : keys) { + Progress progress = progressMap.get(vertexName); + if (progress == null) continue; + + VertexStatus vertexStatus = vertexStatus(statusOptions, vertexName); + if (vertexStatus == null) { + continue; + } + console.printInfo(vertexSummary(vertexName, progress, vertexStatus)); + } + console.printInfo(FILE_HEADER_SEPARATOR); + } + + private String vertexSummary(String vertexName, Progress progress, VertexStatus vertexStatus) { + /* + * Get the CPU & GC + * + * counters org.apache.tez.common.counters.TaskCounter + * GC_TIME_MILLIS=37712 + * CPU_MILLISECONDS=2774230 + */ + final TezCounters vertexCounters = vertexStatus.getVertexCounters(); + final double cpuTimeMillis = getCounterValueByGroupName(vertexCounters, + TaskCounter.class.getName(), + TaskCounter.CPU_MILLISECONDS.name()); + + final double gcTimeMillis = getCounterValueByGroupName(vertexCounters, + TaskCounter.class.getName(), + TaskCounter.GC_TIME_MILLIS.name()); + + /* + * Get the HIVE counters + * + * HIVE + * CREATED_FILES=1 + * DESERIALIZE_ERRORS=0 + * RECORDS_IN_Map_1=550076554 + * RECORDS_OUT_INTERMEDIATE_Map_1=854987 + * RECORDS_OUT_Reducer_2=1 + */ + final long hiveInputRecords = + hiveCounterValue(formattedName(MapOperator.Counter.RECORDS_IN.toString(), vertexName)) + + hiveInputRecordsFromOtherVertices(vertexName); + + final long hiveOutputRecords = + hiveCounterValue(formattedName(FileSinkOperator.Counter.RECORDS_OUT.toString(), vertexName)) + + hiveCounterValue(formattedName(ReduceSinkOperator.Counter.RECORDS_OUT_INTERMEDIATE.toString(), vertexName)); + + final double duration = perfLogger.getDuration(PerfLogger.TEZ_RUN_VERTEX + vertexName); + + return String.format(FORMATTING_PATTERN, + vertexName, + progress.getTotalTaskCount(), + progress.getFailedTaskAttemptCount(), + progress.getKilledTaskAttemptCount(), + secondsFormatter.format((duration)), + commaFormatter.format(cpuTimeMillis), + commaFormatter.format(gcTimeMillis), + commaFormatter.format(hiveInputRecords), + commaFormatter.format(hiveOutputRecords)); + } + + private VertexStatus vertexStatus(Set statusOptions, String vertexName) { + try { + return dagClient.getVertexStatus(vertexName, statusOptions); + } catch (IOException | TezException e) { + // best attempt, shouldn't really kill DAG for this + } + return null; + } + + private void printHeader(SessionState.LogHelper console) { + console.printInfo(FILE_HEADER_SEPARATOR); + console.printInfo(FILE_HEADER); + console.printInfo(FILE_HEADER_SEPARATOR); + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/FSCountersSummary.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/FSCountersSummary.java new file mode 100644 index 0000000..0a28edd --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/FSCountersSummary.java @@ -0,0 +1,92 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.tez.common.counters.FileSystemCounter; +import org.apache.tez.common.counters.TezCounters; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.Progress; +import org.apache.tez.dag.api.client.StatusGetOpts; + +import java.io.IOException; +import java.util.*; + +import static org.apache.hadoop.hive.ql.exec.tez.monitoring.Constants.SEPARATOR; +import static org.apache.hadoop.hive.ql.exec.tez.monitoring.TezJobMonitor.getCounterValueByGroupName; + +public class FSCountersSummary implements PrintSummary { + + private static final String FORMATTING_PATTERN = "%10s %15s %13s %18s %18s %13s"; + private static final String HEADER = String.format(FORMATTING_PATTERN, + "VERTICES", "BYTES_READ", "READ_OPS", "LARGE_READ_OPS", "BYTES_WRITTEN", "WRITE_OPS"); + + private Map progressMap; + private DAGClient dagClient; + + FSCountersSummary(Map progressMap, DAGClient dagClient) { + this.progressMap = progressMap; + this.dagClient = dagClient; + } + + @Override + public void print(SessionState.LogHelper console) { + console.printInfo("FileSystem Counters Summary"); + + SortedSet keys = new TreeSet<>(progressMap.keySet()); + Set statusOptions = new HashSet<>(1); + statusOptions.add(StatusGetOpts.GET_COUNTERS); + // Assuming FileSystem.getAllStatistics() returns all schemes that are accessed on task side + // as well. If not, we need a way to get all the schemes that are accessed by the tez task/llap. + for (FileSystem.Statistics statistics : FileSystem.getAllStatistics()) { + final String scheme = statistics.getScheme().toUpperCase(); + + console.printInfo(""); + console.printInfo("Scheme: " + scheme); + console.printInfo(SEPARATOR); + console.printInfo(HEADER); + console.printInfo(SEPARATOR); + + for (String vertexName : keys) { + TezCounters vertexCounters = vertexCounters(statusOptions, vertexName); + if (vertexCounters != null) { + console.printInfo(summary(scheme, vertexName, vertexCounters)); + } + } + + console.printInfo(SEPARATOR); + } + } + + private String summary(String scheme, String vertexName, TezCounters vertexCounters) { + final String counterGroup = FileSystemCounter.class.getName(); + final long bytesRead = getCounterValueByGroupName(vertexCounters, + counterGroup, scheme + "_" + FileSystemCounter.BYTES_READ.name()); + final long bytesWritten = getCounterValueByGroupName(vertexCounters, + counterGroup, scheme + "_" + FileSystemCounter.BYTES_WRITTEN.name()); + final long readOps = getCounterValueByGroupName(vertexCounters, + counterGroup, scheme + "_" + FileSystemCounter.READ_OPS.name()); + final long largeReadOps = getCounterValueByGroupName(vertexCounters, + counterGroup, scheme + "_" + FileSystemCounter.LARGE_READ_OPS.name()); + final long writeOps = getCounterValueByGroupName(vertexCounters, + counterGroup, scheme + "_" + FileSystemCounter.WRITE_OPS.name()); + + return String.format(FORMATTING_PATTERN, + vertexName, + Utilities.humanReadableByteCount(bytesRead), + readOps, + largeReadOps, + Utilities.humanReadableByteCount(bytesWritten), + writeOps); + } + + private TezCounters vertexCounters(Set statusOptions, String vertexName) { + try { + return dagClient.getVertexStatus(vertexName, statusOptions).getVertexCounters(); + } catch (IOException | TezException e) { + // best attempt, shouldn't really kill DAG for this + } + return null; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/LLAPioSummary.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/LLAPioSummary.java new file mode 100644 index 0000000..81f1755 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/LLAPioSummary.java @@ -0,0 +1,108 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.hive.llap.counters.LlapIOCounters; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.tez.common.counters.TezCounters; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.Progress; +import org.apache.tez.dag.api.client.StatusGetOpts; + +import java.io.IOException; +import java.text.DecimalFormat; +import java.util.*; + +import static org.apache.hadoop.hive.ql.exec.tez.monitoring.Constants.SEPARATOR; +import static org.apache.hadoop.hive.ql.exec.tez.monitoring.TezJobMonitor.getCounterValueByGroupName; + +public class LLAPioSummary implements PrintSummary { + + private static final String LLAP_SUMMARY_HEADER_FORMAT = "%10s %9s %9s %10s %9s %10s %11s %8s %9s"; + private static final String LLAP_IO_SUMMARY_HEADER = "LLAP IO Summary"; + private static final String LLAP_SUMMARY_HEADER = String.format(LLAP_SUMMARY_HEADER_FORMAT, + "VERTICES", "ROWGROUPS", "META_HIT", "META_MISS", "DATA_HIT", "DATA_MISS", + "ALLOCATION", "USED", "TOTAL_IO"); + + + + private final DecimalFormat secondsFormatter = new DecimalFormat("#0.00"); + private Map progressMap; + private DAGClient dagClient; + private boolean first = false; + + LLAPioSummary(Map progressMap, DAGClient dagClient) { + this.progressMap = progressMap; + this.dagClient = dagClient; + } + + @Override + public void print(SessionState.LogHelper console) { + console.printInfo(""); + console.printInfo(LLAP_IO_SUMMARY_HEADER); + + SortedSet keys = new TreeSet<>(progressMap.keySet()); + Set statusOptions = new HashSet<>(1); + statusOptions.add(StatusGetOpts.GET_COUNTERS); + String counterGroup = LlapIOCounters.class.getName(); + for (String vertexName : keys) { + // Reducers do not benefit from LLAP IO so no point in printing + if (vertexName.startsWith("Reducer")) { + continue; + } + TezCounters vertexCounters = vertexCounter(statusOptions, vertexName); + if (vertexCounters != null) { + if (!first) { + console.printInfo(SEPARATOR); + console.printInfo(LLAP_SUMMARY_HEADER); + console.printInfo(SEPARATOR); + first = true; + } + console.printInfo(vertexSummary(vertexName, counterGroup, vertexCounters)); + } + } + console.printInfo(SEPARATOR); + console.printInfo(""); + } + + private String vertexSummary(String vertexName, String counterGroup, TezCounters vertexCounters) { + final long selectedRowgroups = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.SELECTED_ROWGROUPS.name()); + final long metadataCacheHit = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.METADATA_CACHE_HIT.name()); + final long metadataCacheMiss = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.METADATA_CACHE_MISS.name()); + final long cacheHitBytes = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.CACHE_HIT_BYTES.name()); + final long cacheMissBytes = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.CACHE_MISS_BYTES.name()); + final long allocatedBytes = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.ALLOCATED_BYTES.name()); + final long allocatedUsedBytes = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.ALLOCATED_USED_BYTES.name()); + final long totalIoTime = getCounterValueByGroupName(vertexCounters, + counterGroup, LlapIOCounters.TOTAL_IO_TIME_NS.name()); + + + return String.format(LLAP_SUMMARY_HEADER_FORMAT, + vertexName, + selectedRowgroups, + metadataCacheHit, + metadataCacheMiss, + Utilities.humanReadableByteCount(cacheHitBytes), + Utilities.humanReadableByteCount(cacheMissBytes), + Utilities.humanReadableByteCount(allocatedBytes), + Utilities.humanReadableByteCount(allocatedUsedBytes), + secondsFormatter.format(totalIoTime / 1000_000_000.0) + "s"); + } + + private TezCounters vertexCounter(Set statusOptions, String vertexName) { + try { + return dagClient.getVertexStatus(vertexName, statusOptions).getVertexCounters(); + } catch (IOException | TezException e) { + // best attempt, shouldn't really kill DAG for this + } + return null; + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/PrintSummary.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/PrintSummary.java new file mode 100644 index 0000000..6311335 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/PrintSummary.java @@ -0,0 +1,7 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.hive.ql.session.SessionState; + +interface PrintSummary { + void print(SessionState.LogHelper console); +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/QueryExecutionBreakdownSummary.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/QueryExecutionBreakdownSummary.java new file mode 100644 index 0000000..1625ce1 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/QueryExecutionBreakdownSummary.java @@ -0,0 +1,75 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.session.SessionState; + +import java.text.DecimalFormat; + +import static org.apache.hadoop.hive.ql.exec.tez.monitoring.Constants.SEPARATOR; + +class QueryExecutionBreakdownSummary implements PrintSummary { + // Methods summary + private static final String OPERATION_SUMMARY = "%-35s %9s"; + private static final String OPERATION = "OPERATION"; + private static final String DURATION = "DURATION"; + + + private DecimalFormat decimalFormat = new DecimalFormat("#0.00"); + private PerfLogger perfLogger; + + private final Long compileEndTime; + private final Long dagSubmitStartTime; + private final Long submitToRunningDuration; + + QueryExecutionBreakdownSummary(PerfLogger perfLogger) { + this.perfLogger = perfLogger; + this.compileEndTime = perfLogger.getEndTime(PerfLogger.COMPILE); + this.dagSubmitStartTime = perfLogger.getStartTime(PerfLogger.TEZ_SUBMIT_DAG); + this.submitToRunningDuration = perfLogger.getDuration(PerfLogger.TEZ_SUBMIT_TO_RUNNING); + } + + private String formatNumber(long number) { + return decimalFormat.format(number / 1000.0) + "s"; + } + + private String format(String value, long number) { + return String.format(OPERATION_SUMMARY, value, formatNumber(number)); + } + + public void print(SessionState.LogHelper console) { + console.printInfo("Query Execution Summary"); + + String execBreakdownHeader = String.format(OPERATION_SUMMARY, OPERATION, DURATION); + console.printInfo(SEPARATOR); + console.printInfo(execBreakdownHeader); + console.printInfo(SEPARATOR); + + // parse, analyze, optimize and compile + long compile = compileEndTime - perfLogger.getStartTime(PerfLogger.COMPILE); + console.printInfo(format("Compile Query", compile)); + + // prepare plan for submission (building DAG, adding resources, creating scratch dirs etc.) + long totalDAGPrep = dagSubmitStartTime - compileEndTime; + console.printInfo(format("Prepare Plan", totalDAGPrep)); + + // submit to accept dag (if session is closed, this will include re-opening of session time, + // localizing files for AM, submitting DAG) + long submitToAccept = perfLogger.getStartTime(PerfLogger.TEZ_RUN_DAG) - dagSubmitStartTime; + console.printInfo(format("Submit Plan", submitToAccept)); + + // accept to start dag (schedule wait time, resource wait time etc.) + console.printInfo(format("Start DAG", submitToRunningDuration)); + + // time to actually run the dag (actual dag runtime) + final long startToEnd; + if (submitToRunningDuration == 0) { + startToEnd = perfLogger.getDuration(PerfLogger.TEZ_RUN_DAG); + } else { + startToEnd = perfLogger.getEndTime(PerfLogger.TEZ_RUN_DAG) - + perfLogger.getEndTime(PerfLogger.TEZ_SUBMIT_TO_RUNNING); + } + console.printInfo(format("Run DAG", startToEnd)); + console.printInfo(SEPARATOR); + console.printInfo(""); + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java new file mode 100644 index 0000000..bdfc442 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java @@ -0,0 +1,386 @@ +/* + 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.hive.ql.exec.tez.monitoring; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager; +import org.apache.hadoop.hive.ql.log.InPlaceUpdate; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.log.ProgressMonitor; +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; +import org.apache.hive.common.util.ShutdownHookManager; +import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.common.counters.TezCounters; +import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.Progress; +import org.apache.tez.dag.api.client.StatusGetOpts; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.io.StringWriter; +import java.util.*; + +import static org.apache.tez.dag.api.client.DAGStatus.State.RUNNING; + +/** + * TezJobMonitor keeps track of a tez job while it's being executed. It will + * print status to the console and retrieve final status of the job after + * completion. + */ +public class TezJobMonitor { + + private static final String CLASS_NAME = TezJobMonitor.class.getName(); + private static final int CHECK_INTERVAL = 200; + private static final int MAX_RETRY_INTERVAL = 2500; + private static final int PRINT_INTERVAL = 3000; + + private final PerfLogger perfLogger = SessionState.getPerfLogger(); + private static final List shutdownList; + private final Map workMap; + + private transient LogHelper console; + + private long lastPrintTime; + private StringWriter diagnostics = new StringWriter(); + + interface UpdateFunction { + void update(String report); + } + + static { + shutdownList = new LinkedList<>(); + ShutdownHookManager.addShutdownHook(new Runnable() { + @Override + public void run() { + TezJobMonitor.killRunningJobs(); + try { + TezSessionPoolManager.getInstance().closeNonDefaultSessions(false); + } catch (Exception e) { + // ignore + } + } + }); + } + + public static void initShutdownHook() { + Preconditions.checkNotNull(shutdownList, + "Shutdown hook was not properly initialized"); + } + + private final DAGClient dagClient; + private final HiveConf hiveConf; + private final DAG dag; + private final Context context; + private long executionStartTime = 0; + private final UpdateFunction updateFunction; + /** + * Have to use the same instance to render else the number lines printed earlier is lost and the + * screen will print the table again and again. + */ + private final InPlaceUpdate inPlaceUpdate; + + public TezJobMonitor(Map workMap, final DAGClient dagClient, HiveConf conf, DAG dag, + Context ctx) { + this.workMap = workMap; + this.dagClient = dagClient; + this.hiveConf = conf; + this.dag = dag; + this.context = ctx; + console = SessionState.getConsole(); + inPlaceUpdate = new InPlaceUpdate(LogHelper.getInfoStream()); + updateFunction = updateFunction(); + } + + private UpdateFunction updateFunction() { + UpdateFunction logToFileFunction = new UpdateFunction() { + @Override + public void update(String report) { + SessionState.get().updateProgress(progressMonitor()); + console.printInfo(report); + } + }; + UpdateFunction inPlaceUpdateFunction = new UpdateFunction() { + @Override + public void update(String report) { + inPlaceUpdate.render(progressMonitor()); + console.logInfo(report); + } + }; + return InPlaceUpdate.canRenderInPlace(hiveConf) && !SessionState.getConsole().getIsSilent() + ? inPlaceUpdateFunction : logToFileFunction; + } + + private boolean isProfilingEnabled() { + return HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.TEZ_EXEC_SUMMARY) || + Utilities.isPerfOrAboveLogging(hiveConf); + } + + public int monitorExecution() { + boolean done = false; + boolean success = false; + int failedCounter = 0; + int rc = 0; + DAGStatus status = null; + Map progressMap = null; + + + long monitorStartTime = System.currentTimeMillis(); + synchronized (shutdownList) { + shutdownList.add(dagClient); + } + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_DAG); + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_SUBMIT_TO_RUNNING); + DAGStatus.State lastState = null; + String lastReport = null; + boolean running = false; + + while (true) { + + try { + if (context != null) { + context.checkHeartbeaterLockException(); + } + + status = dagClient.getDAGStatus(new HashSet(), CHECK_INTERVAL); + progressMap = status.getVertexProgress(); + DAGStatus.State state = status.getState(); + + if (state != lastState || state == RUNNING) { + lastState = state; + + switch (state) { + case SUBMITTED: + console.printInfo("Status: Submitted"); + break; + case INITING: + console.printInfo("Status: Initializing"); + this.executionStartTime = System.currentTimeMillis(); + break; + case RUNNING: + if (!running) { + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_SUBMIT_TO_RUNNING); + console.printInfo("Status: Running (" + dagClient.getExecutionContext() + ")\n"); + this.executionStartTime = System.currentTimeMillis(); + running = true; + } + lastReport = updateStatus(progressMap, lastReport, updateFunction); + break; + case SUCCEEDED: + if (!running) { + this.executionStartTime = monitorStartTime; + } + lastReport = updateStatus(progressMap, lastReport, updateFunction); + success = true; + running = false; + done = true; + break; + case KILLED: + if (!running) { + this.executionStartTime = monitorStartTime; + } + lastReport = updateStatus(progressMap, lastReport, updateFunction); + console.printInfo("Status: Killed"); + running = false; + done = true; + rc = 1; + break; + case FAILED: + case ERROR: + if (!running) { + this.executionStartTime = monitorStartTime; + } + lastReport = updateStatus(progressMap, lastReport, updateFunction); + console.printError("Status: Failed"); + running = false; + done = true; + rc = 2; + break; + } + } + } catch (Exception e) { + console.printInfo("Exception: " + e.getMessage()); + boolean isInterrupted = hasInterruptedException(e); + if (isInterrupted || (++failedCounter % MAX_RETRY_INTERVAL / CHECK_INTERVAL == 0)) { + try { + console.printInfo("Killing DAG..."); + dagClient.tryKillDAG(); + } catch (IOException | TezException tezException) { + // best effort + } + console.printError("Execution has failed. stack trace: " + Arrays.toString(e.getStackTrace())); + rc = 1; + done = true; + } else { + console.printInfo("Retrying..."); + } + } finally { + if (done) { + if (rc != 0 && status != null) { + for (String diag : status.getDiagnostics()) { + console.printError(diag); + diagnostics.append(diag); + } + } + synchronized (shutdownList) { + shutdownList.remove(dagClient); + } + break; + } + } + } + + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_RUN_DAG); + printSummary(success, progressMap); + return rc; + } + + private void printSummary(boolean success, Map progressMap) { + if (isProfilingEnabled() && success && progressMap != null) { + + double duration = (System.currentTimeMillis() - this.executionStartTime) / 1000.0; + console.printInfo("Status: DAG finished successfully in " + String.format("%.2f seconds", duration)); + console.printInfo(""); + + new QueryExecutionBreakdownSummary(perfLogger).print(console); + new DAGSummary(progressMap, hiveConf, dagClient, dag, perfLogger).print(console); + + //llap IO summary + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.LLAP_IO_ENABLED, false)) { + new LLAPioSummary(progressMap, dagClient).print(console); + new FSCountersSummary(progressMap, dagClient).print(console); + } + console.printInfo(""); + } + } + + private static boolean hasInterruptedException(Throwable e) { + // Hadoop IPC wraps InterruptedException. GRRR. + while (e != null) { + if (e instanceof InterruptedException || e instanceof InterruptedIOException) { + return true; + } + e = e.getCause(); + } + return false; + } + + /** + * killRunningJobs tries to terminate execution of all + * currently running tez queries. No guarantees, best effort only. + */ + private static void killRunningJobs() { + synchronized (shutdownList) { + for (DAGClient c : shutdownList) { + try { + System.err.println("Trying to shutdown DAG"); + c.tryKillDAG(); + } catch (Exception e) { + // ignore + } + } + } + } + + static long getCounterValueByGroupName(TezCounters vertexCounters, String groupNamePattern, + String counterName) { + TezCounter tezCounter = vertexCounters.getGroup(groupNamePattern).findCounter(counterName); + return (tezCounter == null) ? 0 : tezCounter.getValue(); + } + + private String updateStatus(Map progressMap, String lastReport, UpdateFunction function) { + String report = getReport(progressMap); + if (!report.equals(lastReport) || System.currentTimeMillis() >= lastPrintTime + PRINT_INTERVAL) { + function.update(report); + lastPrintTime = System.currentTimeMillis(); + } + return report; + } + + private String getReport(Map progressMap) { + StringBuilder reportBuffer = new StringBuilder(); + + SortedSet keys = new TreeSet(progressMap.keySet()); + for (String s : keys) { + Progress progress = progressMap.get(s); + final int complete = progress.getSucceededTaskCount(); + final int total = progress.getTotalTaskCount(); + final int running = progress.getRunningTaskCount(); + final int failed = progress.getFailedTaskAttemptCount(); + if (total <= 0) { + reportBuffer.append(String.format("%s: -/-\t", s)); + } else { + if (complete == total) { + /* + * We may have missed the start of the vertex due to the 3 seconds interval + */ + if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) { + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); + } + + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); + } + if (complete < total && (complete > 0 || running > 0 || failed > 0)) { + + if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) { + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s); + } + + /* vertex is started, but not complete */ + if (failed > 0) { + reportBuffer.append(String.format("%s: %d(+%d,-%d)/%d\t", s, complete, running, failed, total)); + } else { + reportBuffer.append(String.format("%s: %d(+%d)/%d\t", s, complete, running, total)); + } + } else { + /* vertex is waiting for input/slots or complete */ + if (failed > 0) { + /* tasks finished but some failed */ + reportBuffer.append(String.format("%s: %d(-%d)/%d\t", s, complete, failed, total)); + } else { + reportBuffer.append(String.format("%s: %d/%d\t", s, complete, total)); + } + } + } + } + + return reportBuffer.toString(); + } + + public String getDiagnostics() { + return diagnostics.toString(); + } + + public ProgressMonitor progressMonitor() { + try { + return new TezProgressMonitor(dagClient, workMap, console, executionStartTime); + } catch (IOException | TezException e) { + console.printInfo("Getting Progress Information: " + e.getMessage() + " stack trace: " + + Arrays.toString(e.getStackTrace())); + } + return TezProgressMonitor.NULL; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezProgressMonitor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezProgressMonitor.java new file mode 100644 index 0000000..069bb74 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezProgressMonitor.java @@ -0,0 +1,315 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.hive.ql.log.ProgressMonitor; +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.Progress; +import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.dag.api.client.VertexStatus; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.apache.tez.dag.api.client.DAGStatus.State.KILLED; + +class TezProgressMonitor implements ProgressMonitor { + private static final int COLUMN_1_WIDTH = 16; + private final Map workMap; + private final SessionState.LogHelper console; + private final long executionStartTime; + private final DAGStatus status; + Map vertexStatusMap = new HashMap<>(); + Map progressCountsMap = new HashMap<>(); + + /** + * Try to get most the data required from dagClient in the constructor itself so that even after + * the tez job has finished this object can be used for later use.s + */ + TezProgressMonitor(DAGClient dagClient, Map workMap, + SessionState.LogHelper console, long executionStartTime) throws IOException, TezException { + status = dagClient.getDAGStatus(new HashSet()); + Map progressMap = status.getVertexProgress(); + this.workMap = workMap; + this.console = console; + this.executionStartTime = executionStartTime; + for (Map.Entry entry : progressMap.entrySet()) { + String vertexName = entry.getKey(); + progressCountsMap + .put(vertexName, new StateFullProgress(entry.getValue(), status.getState())); + try { + vertexStatusMap.put(vertexName, dagClient.getVertexStatus(vertexName, null)); + } catch (IOException e) { + // best attempt, shouldn't really kill DAG for this + } + } + } + + public List headers() { + return Arrays.asList( + "VERTICES", + "MODE", + "STATUS", + "TOTAL", + "COMPLETED", + "RUNNING", + "PENDING", + "FAILED", + "KILLED" + ); + } + + public List> rows() { + try { + List> results = new ArrayList<>(); + SortedSet keys = new TreeSet<>(progressCountsMap.keySet()); + for (String s : keys) { + StateFullProgress progress = progressCountsMap.get(s); + + // Map 1 .......... container SUCCEEDED 7 7 0 0 0 0 + + results.add( + Arrays.asList( + getNameWithProgress(s, progress.succeededTaskCount, progress.totalTaskCount), + getMode(s, workMap), + progress.vertexStatus(vertexStatusMap.get(s)), + progress.total(), + progress.completed(), + progress.running(), + progress.pending(), + progress.failed(), + progress.killed() + ) + ); + } + return results; + } catch (Exception e) { + console.printInfo( + "Getting Progress Bar table rows failed: " + e.getMessage() + " stack trace: " + Arrays + .toString(e.getStackTrace()) + ); + } + return Collections.emptyList(); + } + + // ------------------------------------------------------------------------------- + // VERTICES: 03/04 [=================>>-----] 86% ELAPSED TIME: 1.71 s + // ------------------------------------------------------------------------------- + // contains footerSummary , progressedPercentage, starTime + + @Override + public String footerSummary() { + return String.format("VERTICES: %02d/%02d", completed(), progressCountsMap.keySet().size()); + } + + @Override + public long startTime() { + return executionStartTime; + } + + @Override + public double progressedPercentage() { + int sumTotal = 0, sumComplete = 0; + for (String s : progressCountsMap.keySet()) { + StateFullProgress progress = progressCountsMap.get(s); + final int complete = progress.succeededTaskCount; + final int total = progress.totalTaskCount; + if (total > 0) { + sumTotal += total; + sumComplete += complete; + } + } + return (sumTotal == 0) ? 0.0f : (float) sumComplete / (float) sumTotal; + } + + @Override + public String executionStatus() { + return this.status.getState().name(); + } + + private int completed() { + Set completed = new HashSet<>(); + for (String s : progressCountsMap.keySet()) { + StateFullProgress progress = progressCountsMap.get(s); + final int complete = progress.succeededTaskCount; + final int total = progress.totalTaskCount; + if (total > 0) { + if (complete == total) { + completed.add(s); + } + } + } + return completed.size(); + } + + // Map 1 .......... + + private String getNameWithProgress(String s, int complete, int total) { + String result = ""; + if (s != null) { + float percent = total == 0 ? 0.0f : (float) complete / (float) total; + // lets use the remaining space in column 1 as progress bar + int spaceRemaining = COLUMN_1_WIDTH - s.length() - 1; + String trimmedVName = s; + + // if the vertex name is longer than column 1 width, trim it down + // "Tez Merge File Work" will become "Tez Merge File.." + if (s.length() > COLUMN_1_WIDTH) { + trimmedVName = s.substring(0, COLUMN_1_WIDTH - 1); + trimmedVName = trimmedVName + ".."; + } + + result = trimmedVName + " "; + int toFill = (int) (spaceRemaining * percent); + for (int i = 0; i < toFill; i++) { + result += "."; + } + } + return result; + } + + private String getMode(String name, Map workMap) { + String mode = "container"; + BaseWork work = workMap.get(name); + if (work != null) { + // uber > llap > container + if (work.getUberMode()) { + mode = "uber"; + } else if (work.getLlapMode()) { + mode = "llap"; + } else { + mode = "container"; + } + } + return mode; + } + + static class StateFullProgress { + private final int totalTaskCount; + private final int succeededTaskCount; + private final int failedTaskAttemptCount; + private final long killedTaskAttemptCount; + private final int runningTaskCount; + private final DAGStatus.State dagState; + + StateFullProgress(Progress progress, DAGStatus.State dagState) { + this(progress.getTotalTaskCount(), progress.getSucceededTaskCount(), + progress.getFailedTaskAttemptCount(), progress.getKilledTaskAttemptCount(), + progress.getRunningTaskCount(), dagState); + } + + StateFullProgress(int totalTaskCount, int succeededTaskCount, int failedTaskAttemptCount, + int killedTaskAttemptCount, int runningTaskCount, DAGStatus.State dagState) { + this.totalTaskCount = totalTaskCount; + this.succeededTaskCount = succeededTaskCount; + this.failedTaskAttemptCount = failedTaskAttemptCount; + this.killedTaskAttemptCount = killedTaskAttemptCount; + this.runningTaskCount = runningTaskCount; + this.dagState = dagState; + } + + boolean isRunning() { + return succeededTaskCount < totalTaskCount && (succeededTaskCount > 0 || runningTaskCount > 0 + || failedTaskAttemptCount > 0); + } + + String vertexStatus(VertexStatus vertexStatus) { + // To get vertex status we can use DAGClient.getVertexStatus(), but it will be expensive to + // get status from AM for every refresh of the UI. Lets infer the state from task counts. + // Only if DAG is FAILED or KILLED the vertex status is fetched from AM. + VertexStatus.State vertexState = VertexStatus.State.INITIALIZING; + if (totalTaskCount > 0) { + vertexState = VertexStatus.State.INITED; + } + + // RUNNING state + if (isRunning()) { + vertexState = VertexStatus.State.RUNNING; + } + + // SUCCEEDED state + if (succeededTaskCount == totalTaskCount) { + vertexState = VertexStatus.State.SUCCEEDED; + } + + // DAG might have been killed, lets try to get vertex state from AM before dying + // KILLED or FAILED state + if (dagState == KILLED) { + if (vertexStatus != null) { + vertexState = vertexStatus.getState(); + } + } + return vertexState.toString(); + } + + // "TOTAL", "COMPLETED", "RUNNING", "PENDING", "FAILED", "KILLED" + + String total() { + return String.valueOf(totalTaskCount); + } + + String completed() { + return String.valueOf(succeededTaskCount); + } + + String running() { + return String.valueOf(runningTaskCount); + } + + String pending() { + return String.valueOf(totalTaskCount - succeededTaskCount - runningTaskCount); + } + + String failed() { + return String.valueOf(failedTaskAttemptCount); + } + + String killed() { + return String.valueOf(killedTaskAttemptCount); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + StateFullProgress that = (StateFullProgress) o; + + if (totalTaskCount != that.totalTaskCount) + return false; + if (succeededTaskCount != that.succeededTaskCount) + return false; + if (failedTaskAttemptCount != that.failedTaskAttemptCount) + return false; + if (killedTaskAttemptCount != that.killedTaskAttemptCount) + return false; + if (runningTaskCount != that.runningTaskCount) + return false; + return dagState == that.dagState; + } + + @Override + public int hashCode() { + int result = totalTaskCount; + result = 31 * result + succeededTaskCount; + result = 31 * result + failedTaskAttemptCount; + result = 31 * result + (int) (killedTaskAttemptCount ^ (killedTaskAttemptCount >>> 32)); + result = 31 * result + runningTaskCount; + result = 31 * result + dagState.hashCode(); + return result; + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index c5b3517..5ff5ffe 100644 --- ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -127,11 +127,11 @@ import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.FunctionTask; import org.apache.hadoop.hive.ql.exec.FunctionUtils; -import org.apache.hadoop.hive.ql.exec.InPlaceUpdates; import org.apache.hadoop.hive.ql.exec.SerializationUtilities; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.index.HiveIndexHandler; import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.log.InPlaceUpdate; import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPrunerUtils; import org.apache.hadoop.hive.ql.plan.AddPartitionDesc; @@ -1898,7 +1898,7 @@ private void constructOneLBLocationMap(FileStatus fSta, final AtomicInteger partitionsLoaded = new AtomicInteger(0); final boolean inPlaceEligible = conf.getLong("fs.trash.interval", 0) <= 0 - && InPlaceUpdates.inPlaceEligible(conf); + && InPlaceUpdate.canRenderInPlace(conf) && !SessionState.getConsole().getIsSilent(); final PrintStream ps = (inPlaceEligible) ? SessionState.getConsole().getInfoStream() : null; final SessionState parentSession = SessionState.get(); @@ -1926,9 +1926,9 @@ public Void call() throws Exception { if (inPlaceEligible) { synchronized (ps) { - InPlaceUpdates.rePositionCursor(ps); + InPlaceUpdate.rePositionCursor(ps); partitionsLoaded.incrementAndGet(); - InPlaceUpdates.reprintLine(ps, "Loaded : " + partitionsLoaded.get() + "/" + InPlaceUpdate.reprintLine(ps, "Loaded : " + partitionsLoaded.get() + "/" + partsToLoad + " partitions."); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 453e0a5..992e715 100644 --- ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -69,6 +69,7 @@ import org.apache.hadoop.hive.ql.lockmgr.LockException; import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory; import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.log.ProgressMonitor; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveUtils; @@ -185,6 +186,7 @@ private HiveAuthorizationProvider authorizer; private HiveAuthorizer authorizerV2; + private ProgressMonitor progressMonitor; public enum AuthorizationMode{V1, V2}; @@ -1558,6 +1560,7 @@ public void close() throws IOException { // removes the threadlocal variables, closes underlying HMS connection Hive.closeCurrent(); } + progressMonitor = null; } private void unCacheDataNucleusClassLoaders() { @@ -1738,6 +1741,15 @@ public void setForwardedAddresses(List forwardedAddresses) { public String getReloadableAuxJars() { return StringUtils.join(preReloadableAuxJars, ','); } + + public void updateProgress(ProgressMonitor progressMonitor) { + this.progressMonitor = progressMonitor; + } + + public ProgressMonitor progressMonitor() { + return progressMonitor; + } + } class ResourceMaps { diff --git ql/src/test/org/apache/hadoop/hive/ql/exec/tez/monitoring/TestTezProgressMonitor.java ql/src/test/org/apache/hadoop/hive/ql/exec/tez/monitoring/TestTezProgressMonitor.java new file mode 100644 index 0000000..96b28f7 --- /dev/null +++ ql/src/test/org/apache/hadoop/hive/ql/exec/tez/monitoring/TestTezProgressMonitor.java @@ -0,0 +1,100 @@ +package org.apache.hadoop.hive.ql.exec.tez.monitoring; + +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.Progress; +import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.dag.api.client.VertexStatus; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.anySet; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class TestTezProgressMonitor { + + private static final String REDUCER = "Reducer"; + private static final String MAPPER = "Mapper"; + @Mock + private DAGClient dagClient; + @Mock + private SessionState.LogHelper console; + @Mock + private DAGStatus dagStatus; + @Mock + private Progress mapperProgress; + @Mock + private Progress reducerProgress; + @Mock + private VertexStatus succeeded; + @Mock + private VertexStatus running; + + private Map progressMap() { + return new HashMap() {{ + put(MAPPER, setup(mapperProgress, 2, 1, 3, 4, 5)); + put(REDUCER, setup(reducerProgress, 3, 2, 1, 0, 1)); + }}; + } + + private Progress setup(Progress progressMock, int total, int succeeded, int failedAttempt, + int killedAttempt, int running) { + when(progressMock.getTotalTaskCount()).thenReturn(total); + when(progressMock.getSucceededTaskCount()).thenReturn(succeeded); + when(progressMock.getFailedTaskAttemptCount()).thenReturn(failedAttempt); + when(progressMock.getKilledTaskAttemptCount()).thenReturn(killedAttempt); + when(progressMock.getRunningTaskCount()).thenReturn(running); + return progressMock; + } + + @Test + public void setupInternalStateOnObjectCreation() throws IOException, TezException { + + when(dagClient.getDAGStatus(Matchers.>any())).thenReturn(dagStatus); + Map t = progressMap(); + when(dagStatus.getVertexProgress()).thenReturn(t); + when(dagStatus.getState()).thenReturn(DAGStatus.State.RUNNING); + when(dagClient.getVertexStatus(eq(MAPPER), anySet())).thenReturn(succeeded); + when(dagClient.getVertexStatus(eq(REDUCER), anySet())).thenReturn(running); + + TezProgressMonitor monitor = + new TezProgressMonitor(dagClient, new HashMap(), console, Long.MAX_VALUE); + + verify(dagClient).getVertexStatus(eq(MAPPER), isNull(Set.class)); + verify(dagClient).getVertexStatus(eq(REDUCER), isNull(Set.class)); + + assertThat(monitor.vertexStatusMap.keySet(), hasItems(MAPPER, REDUCER)); + assertThat(monitor.vertexStatusMap.get(MAPPER), is(sameInstance(succeeded))); + assertThat(monitor.vertexStatusMap.get(REDUCER), is(sameInstance(running))); + + assertThat(monitor.progressCountsMap.keySet(), hasItems(MAPPER, REDUCER)); + TezProgressMonitor.StateFullProgress expectedMapperState = + new TezProgressMonitor.StateFullProgress(2, 1, 3, 4, 5, DAGStatus.State.RUNNING); + assertThat(monitor.progressCountsMap.get(MAPPER), is(equalTo(expectedMapperState))); + + TezProgressMonitor.StateFullProgress expectedReducerState = + new TezProgressMonitor.StateFullProgress(3, 2, 1, 0, 1, DAGStatus.State.RUNNING); + assertThat(monitor.progressCountsMap.get(REDUCER), is(equalTo(expectedReducerState))); + } + +} \ No newline at end of file diff --git service-rpc/if/TCLIService.thrift service-rpc/if/TCLIService.thrift index a4fa7b0..66a78fb 100644 --- service-rpc/if/TCLIService.thrift +++ service-rpc/if/TCLIService.thrift @@ -1202,6 +1202,31 @@ struct TRenewDelegationTokenResp { 1: required TStatus status } +struct TProgressUpdateReq { + // Operation against which to get the progress bar update + 1: required TOperationHandle operationHandle +} + +enum TJobExecutionStatus { + SUBMITTED, + INITING, + RUNNING, + SUCCEEDED, + KILLED, + FAILED, + ERROR, + NOT_AVAILABLE +} + +struct TProgressUpdateResp { + 1: required list headerNames + 2: required list> rows + 3: required double progressedPercentage + 4: required TJobExecutionStatus status + 5: required string footerSummary + 6: required i64 startTime +} + service TCLIService { TOpenSessionResp OpenSession(1:TOpenSessionReq req); @@ -1245,4 +1270,6 @@ service TCLIService { TCancelDelegationTokenResp CancelDelegationToken(1:TCancelDelegationTokenReq req); TRenewDelegationTokenResp RenewDelegationToken(1:TRenewDelegationTokenReq req); + + TProgressUpdateResp GetProgressUpdate(1:TProgressUpdateReq req); } diff --git service-rpc/src/gen/thrift/gen-cpp/TCLIService.cpp service-rpc/src/gen/thrift/gen-cpp/TCLIService.cpp index 3597d44..dfc6e26 100644 --- service-rpc/src/gen/thrift/gen-cpp/TCLIService.cpp +++ service-rpc/src/gen/thrift/gen-cpp/TCLIService.cpp @@ -3935,6 +3935,193 @@ uint32_t TCLIService_RenewDelegationToken_presult::read(::apache::thrift::protoc return xfer; } + +TCLIService_GetProgressUpdate_args::~TCLIService_GetProgressUpdate_args() throw() { +} + + +uint32_t TCLIService_GetProgressUpdate_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->req.read(iprot); + this->__isset.req = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t TCLIService_GetProgressUpdate_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TCLIService_GetProgressUpdate_args"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->req.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +TCLIService_GetProgressUpdate_pargs::~TCLIService_GetProgressUpdate_pargs() throw() { +} + + +uint32_t TCLIService_GetProgressUpdate_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TCLIService_GetProgressUpdate_pargs"); + + xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->req)).write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +TCLIService_GetProgressUpdate_result::~TCLIService_GetProgressUpdate_result() throw() { +} + + +uint32_t TCLIService_GetProgressUpdate_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->success.read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + +uint32_t TCLIService_GetProgressUpdate_result::write(::apache::thrift::protocol::TProtocol* oprot) const { + + uint32_t xfer = 0; + + xfer += oprot->writeStructBegin("TCLIService_GetProgressUpdate_result"); + + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); + } + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + + +TCLIService_GetProgressUpdate_presult::~TCLIService_GetProgressUpdate_presult() throw() { +} + + +uint32_t TCLIService_GetProgressUpdate_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 0: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += (*(this->success)).read(iprot); + this->__isset.success = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + return xfer; +} + void TCLIServiceClient::OpenSession(TOpenSessionResp& _return, const TOpenSessionReq& req) { send_OpenSession(req); @@ -5153,6 +5340,64 @@ void TCLIServiceClient::recv_RenewDelegationToken(TRenewDelegationTokenResp& _re throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "RenewDelegationToken failed: unknown result"); } +void TCLIServiceClient::GetProgressUpdate(TProgressUpdateResp& _return, const TProgressUpdateReq& req) +{ + send_GetProgressUpdate(req); + recv_GetProgressUpdate(_return); +} + +void TCLIServiceClient::send_GetProgressUpdate(const TProgressUpdateReq& req) +{ + int32_t cseqid = 0; + oprot_->writeMessageBegin("GetProgressUpdate", ::apache::thrift::protocol::T_CALL, cseqid); + + TCLIService_GetProgressUpdate_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); +} + +void TCLIServiceClient::recv_GetProgressUpdate(TProgressUpdateResp& _return) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + iprot_->readMessageBegin(fname, mtype, rseqid); + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("GetProgressUpdate") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + TCLIService_GetProgressUpdate_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + return; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "GetProgressUpdate failed: unknown result"); +} + bool TCLIServiceProcessor::dispatchCall(::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, const std::string& fname, int32_t seqid, void* callContext) { ProcessMap::iterator pfn; pfn = processMap_.find(fname); @@ -6306,6 +6551,60 @@ void TCLIServiceProcessor::process_RenewDelegationToken(int32_t seqid, ::apache: } } +void TCLIServiceProcessor::process_GetProgressUpdate(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +{ + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("TCLIService.GetProgressUpdate", callContext); + } + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "TCLIService.GetProgressUpdate"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "TCLIService.GetProgressUpdate"); + } + + TCLIService_GetProgressUpdate_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "TCLIService.GetProgressUpdate", bytes); + } + + TCLIService_GetProgressUpdate_result result; + try { + iface_->GetProgressUpdate(result.success, args.req); + result.__isset.success = true; + } catch (const std::exception& e) { + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->handlerError(ctx, "TCLIService.GetProgressUpdate"); + } + + ::apache::thrift::TApplicationException x(e.what()); + oprot->writeMessageBegin("GetProgressUpdate", ::apache::thrift::protocol::T_EXCEPTION, seqid); + x.write(oprot); + oprot->writeMessageEnd(); + oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + return; + } + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "TCLIService.GetProgressUpdate"); + } + + oprot->writeMessageBegin("GetProgressUpdate", ::apache::thrift::protocol::T_REPLY, seqid); + result.write(oprot); + oprot->writeMessageEnd(); + bytes = oprot->getTransport()->writeEnd(); + oprot->getTransport()->flush(); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postWrite(ctx, "TCLIService.GetProgressUpdate", bytes); + } +} + ::boost::shared_ptr< ::apache::thrift::TProcessor > TCLIServiceProcessorFactory::getProcessor(const ::apache::thrift::TConnectionInfo& connInfo) { ::apache::thrift::ReleaseHandler< TCLIServiceIfFactory > cleanup(handlerFactory_); ::boost::shared_ptr< TCLIServiceIf > handler(handlerFactory_->getHandler(connInfo), cleanup); @@ -8077,5 +8376,89 @@ void TCLIServiceConcurrentClient::recv_RenewDelegationToken(TRenewDelegationToke } // end while(true) } +void TCLIServiceConcurrentClient::GetProgressUpdate(TProgressUpdateResp& _return, const TProgressUpdateReq& req) +{ + int32_t seqid = send_GetProgressUpdate(req); + recv_GetProgressUpdate(_return, seqid); +} + +int32_t TCLIServiceConcurrentClient::send_GetProgressUpdate(const TProgressUpdateReq& req) +{ + int32_t cseqid = this->sync_.generateSeqId(); + ::apache::thrift::async::TConcurrentSendSentry sentry(&this->sync_); + oprot_->writeMessageBegin("GetProgressUpdate", ::apache::thrift::protocol::T_CALL, cseqid); + + TCLIService_GetProgressUpdate_pargs args; + args.req = &req; + args.write(oprot_); + + oprot_->writeMessageEnd(); + oprot_->getTransport()->writeEnd(); + oprot_->getTransport()->flush(); + + sentry.commit(); + return cseqid; +} + +void TCLIServiceConcurrentClient::recv_GetProgressUpdate(TProgressUpdateResp& _return, const int32_t seqid) +{ + + int32_t rseqid = 0; + std::string fname; + ::apache::thrift::protocol::TMessageType mtype; + + // the read mutex gets dropped and reacquired as part of waitForWork() + // The destructor of this sentry wakes up other clients + ::apache::thrift::async::TConcurrentRecvSentry sentry(&this->sync_, seqid); + + while(true) { + if(!this->sync_.getPending(fname, mtype, rseqid)) { + iprot_->readMessageBegin(fname, mtype, rseqid); + } + if(seqid == rseqid) { + if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { + ::apache::thrift::TApplicationException x; + x.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + sentry.commit(); + throw x; + } + if (mtype != ::apache::thrift::protocol::T_REPLY) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + } + if (fname.compare("GetProgressUpdate") != 0) { + iprot_->skip(::apache::thrift::protocol::T_STRUCT); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + // in a bad state, don't commit + using ::apache::thrift::protocol::TProtocolException; + throw TProtocolException(TProtocolException::INVALID_DATA); + } + TCLIService_GetProgressUpdate_presult result; + result.success = &_return; + result.read(iprot_); + iprot_->readMessageEnd(); + iprot_->getTransport()->readEnd(); + + if (result.__isset.success) { + // _return pointer has now been filled + sentry.commit(); + return; + } + // in a bad state, don't commit + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "GetProgressUpdate failed: unknown result"); + } + // seqid != rseqid + this->sync_.updatePending(fname, mtype, rseqid); + + // this will temporarily unlock the readMutex, and let other clients get work done + this->sync_.waitForWork(seqid); + } // end while(true) +} + }}}}} // namespace diff --git service-rpc/src/gen/thrift/gen-cpp/TCLIService.h service-rpc/src/gen/thrift/gen-cpp/TCLIService.h index 5fd423d..77a55a1 100644 --- service-rpc/src/gen/thrift/gen-cpp/TCLIService.h +++ service-rpc/src/gen/thrift/gen-cpp/TCLIService.h @@ -42,6 +42,7 @@ class TCLIServiceIf { virtual void GetDelegationToken(TGetDelegationTokenResp& _return, const TGetDelegationTokenReq& req) = 0; virtual void CancelDelegationToken(TCancelDelegationTokenResp& _return, const TCancelDelegationTokenReq& req) = 0; virtual void RenewDelegationToken(TRenewDelegationTokenResp& _return, const TRenewDelegationTokenReq& req) = 0; + virtual void GetProgressUpdate(TProgressUpdateResp& _return, const TProgressUpdateReq& req) = 0; }; class TCLIServiceIfFactory { @@ -134,6 +135,9 @@ class TCLIServiceNull : virtual public TCLIServiceIf { void RenewDelegationToken(TRenewDelegationTokenResp& /* _return */, const TRenewDelegationTokenReq& /* req */) { return; } + void GetProgressUpdate(TProgressUpdateResp& /* _return */, const TProgressUpdateReq& /* req */) { + return; + } }; typedef struct _TCLIService_OpenSession_args__isset { @@ -2320,6 +2324,110 @@ class TCLIService_RenewDelegationToken_presult { }; +typedef struct _TCLIService_GetProgressUpdate_args__isset { + _TCLIService_GetProgressUpdate_args__isset() : req(false) {} + bool req :1; +} _TCLIService_GetProgressUpdate_args__isset; + +class TCLIService_GetProgressUpdate_args { + public: + + TCLIService_GetProgressUpdate_args(const TCLIService_GetProgressUpdate_args&); + TCLIService_GetProgressUpdate_args& operator=(const TCLIService_GetProgressUpdate_args&); + TCLIService_GetProgressUpdate_args() { + } + + virtual ~TCLIService_GetProgressUpdate_args() throw(); + TProgressUpdateReq req; + + _TCLIService_GetProgressUpdate_args__isset __isset; + + void __set_req(const TProgressUpdateReq& val); + + bool operator == (const TCLIService_GetProgressUpdate_args & rhs) const + { + if (!(req == rhs.req)) + return false; + return true; + } + bool operator != (const TCLIService_GetProgressUpdate_args &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TCLIService_GetProgressUpdate_args & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class TCLIService_GetProgressUpdate_pargs { + public: + + + virtual ~TCLIService_GetProgressUpdate_pargs() throw(); + const TProgressUpdateReq* req; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _TCLIService_GetProgressUpdate_result__isset { + _TCLIService_GetProgressUpdate_result__isset() : success(false) {} + bool success :1; +} _TCLIService_GetProgressUpdate_result__isset; + +class TCLIService_GetProgressUpdate_result { + public: + + TCLIService_GetProgressUpdate_result(const TCLIService_GetProgressUpdate_result&); + TCLIService_GetProgressUpdate_result& operator=(const TCLIService_GetProgressUpdate_result&); + TCLIService_GetProgressUpdate_result() { + } + + virtual ~TCLIService_GetProgressUpdate_result() throw(); + TProgressUpdateResp success; + + _TCLIService_GetProgressUpdate_result__isset __isset; + + void __set_success(const TProgressUpdateResp& val); + + bool operator == (const TCLIService_GetProgressUpdate_result & rhs) const + { + if (!(success == rhs.success)) + return false; + return true; + } + bool operator != (const TCLIService_GetProgressUpdate_result &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TCLIService_GetProgressUpdate_result & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +typedef struct _TCLIService_GetProgressUpdate_presult__isset { + _TCLIService_GetProgressUpdate_presult__isset() : success(false) {} + bool success :1; +} _TCLIService_GetProgressUpdate_presult__isset; + +class TCLIService_GetProgressUpdate_presult { + public: + + + virtual ~TCLIService_GetProgressUpdate_presult() throw(); + TProgressUpdateResp* success; + + _TCLIService_GetProgressUpdate_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + class TCLIServiceClient : virtual public TCLIServiceIf { public: TCLIServiceClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) { @@ -2408,6 +2516,9 @@ class TCLIServiceClient : virtual public TCLIServiceIf { void RenewDelegationToken(TRenewDelegationTokenResp& _return, const TRenewDelegationTokenReq& req); void send_RenewDelegationToken(const TRenewDelegationTokenReq& req); void recv_RenewDelegationToken(TRenewDelegationTokenResp& _return); + void GetProgressUpdate(TProgressUpdateResp& _return, const TProgressUpdateReq& req); + void send_GetProgressUpdate(const TProgressUpdateReq& req); + void recv_GetProgressUpdate(TProgressUpdateResp& _return); protected: boost::shared_ptr< ::apache::thrift::protocol::TProtocol> piprot_; boost::shared_ptr< ::apache::thrift::protocol::TProtocol> poprot_; @@ -2444,6 +2555,7 @@ class TCLIServiceProcessor : public ::apache::thrift::TDispatchProcessor { void process_GetDelegationToken(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); void process_CancelDelegationToken(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); void process_RenewDelegationToken(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); + void process_GetProgressUpdate(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext); public: TCLIServiceProcessor(boost::shared_ptr iface) : iface_(iface) { @@ -2468,6 +2580,7 @@ class TCLIServiceProcessor : public ::apache::thrift::TDispatchProcessor { processMap_["GetDelegationToken"] = &TCLIServiceProcessor::process_GetDelegationToken; processMap_["CancelDelegationToken"] = &TCLIServiceProcessor::process_CancelDelegationToken; processMap_["RenewDelegationToken"] = &TCLIServiceProcessor::process_RenewDelegationToken; + processMap_["GetProgressUpdate"] = &TCLIServiceProcessor::process_GetProgressUpdate; } virtual ~TCLIServiceProcessor() {} @@ -2706,6 +2819,16 @@ class TCLIServiceMultiface : virtual public TCLIServiceIf { return; } + void GetProgressUpdate(TProgressUpdateResp& _return, const TProgressUpdateReq& req) { + size_t sz = ifaces_.size(); + size_t i = 0; + for (; i < (sz - 1); ++i) { + ifaces_[i]->GetProgressUpdate(_return, req); + } + ifaces_[i]->GetProgressUpdate(_return, req); + return; + } + }; // The 'concurrent' client is a thread safe client that correctly handles @@ -2799,6 +2922,9 @@ class TCLIServiceConcurrentClient : virtual public TCLIServiceIf { void RenewDelegationToken(TRenewDelegationTokenResp& _return, const TRenewDelegationTokenReq& req); int32_t send_RenewDelegationToken(const TRenewDelegationTokenReq& req); void recv_RenewDelegationToken(TRenewDelegationTokenResp& _return, const int32_t seqid); + void GetProgressUpdate(TProgressUpdateResp& _return, const TProgressUpdateReq& req); + int32_t send_GetProgressUpdate(const TProgressUpdateReq& req); + void recv_GetProgressUpdate(TProgressUpdateResp& _return, const int32_t seqid); protected: boost::shared_ptr< ::apache::thrift::protocol::TProtocol> piprot_; boost::shared_ptr< ::apache::thrift::protocol::TProtocol> poprot_; diff --git service-rpc/src/gen/thrift/gen-cpp/TCLIService_server.skeleton.cpp service-rpc/src/gen/thrift/gen-cpp/TCLIService_server.skeleton.cpp index 5d7caf9..743b792 100644 --- service-rpc/src/gen/thrift/gen-cpp/TCLIService_server.skeleton.cpp +++ service-rpc/src/gen/thrift/gen-cpp/TCLIService_server.skeleton.cpp @@ -127,6 +127,11 @@ class TCLIServiceHandler : virtual public TCLIServiceIf { printf("RenewDelegationToken\n"); } + void GetProgressUpdate(TProgressUpdateResp& _return, const TProgressUpdateReq& req) { + // Your implementation goes here + printf("GetProgressUpdate\n"); + } + }; int main(int argc, char **argv) { diff --git service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp index 2f460e8..35ac180 100644 --- service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp +++ service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.cpp @@ -269,6 +269,28 @@ const char* _kTFetchOrientationNames[] = { }; const std::map _TFetchOrientation_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(6, _kTFetchOrientationValues, _kTFetchOrientationNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); +int _kTJobExecutionStatusValues[] = { + TJobExecutionStatus::SUBMITTED, + TJobExecutionStatus::INITING, + TJobExecutionStatus::RUNNING, + TJobExecutionStatus::SUCCEEDED, + TJobExecutionStatus::KILLED, + TJobExecutionStatus::FAILED, + TJobExecutionStatus::ERROR, + TJobExecutionStatus::NOT_AVAILABLE +}; +const char* _kTJobExecutionStatusNames[] = { + "SUBMITTED", + "INITING", + "RUNNING", + "SUCCEEDED", + "KILLED", + "FAILED", + "ERROR", + "NOT_AVAILABLE" +}; +const std::map _TJobExecutionStatus_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, _kTJobExecutionStatusValues, _kTJobExecutionStatusNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); + TTypeQualifierValue::~TTypeQualifierValue() throw() { } @@ -9984,4 +10006,353 @@ void TRenewDelegationTokenResp::printTo(std::ostream& out) const { out << ")"; } + +TProgressUpdateReq::~TProgressUpdateReq() throw() { +} + + +void TProgressUpdateReq::__set_operationHandle(const TOperationHandle& val) { + this->operationHandle = val; +} + +uint32_t TProgressUpdateReq::read(::apache::thrift::protocol::TProtocol* iprot) { + + apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + bool isset_operationHandle = false; + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_STRUCT) { + xfer += this->operationHandle.read(iprot); + isset_operationHandle = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_operationHandle) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t TProgressUpdateReq::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TProgressUpdateReq"); + + xfer += oprot->writeFieldBegin("operationHandle", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->operationHandle.write(oprot); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TProgressUpdateReq &a, TProgressUpdateReq &b) { + using ::std::swap; + swap(a.operationHandle, b.operationHandle); +} + +TProgressUpdateReq::TProgressUpdateReq(const TProgressUpdateReq& other302) { + operationHandle = other302.operationHandle; +} +TProgressUpdateReq& TProgressUpdateReq::operator=(const TProgressUpdateReq& other303) { + operationHandle = other303.operationHandle; + return *this; +} +void TProgressUpdateReq::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TProgressUpdateReq("; + out << "operationHandle=" << to_string(operationHandle); + out << ")"; +} + + +TProgressUpdateResp::~TProgressUpdateResp() throw() { +} + + +void TProgressUpdateResp::__set_headerNames(const std::vector & val) { + this->headerNames = val; +} + +void TProgressUpdateResp::__set_rows(const std::vector > & val) { + this->rows = val; +} + +void TProgressUpdateResp::__set_progressedPercentage(const double val) { + this->progressedPercentage = val; +} + +void TProgressUpdateResp::__set_status(const TJobExecutionStatus::type val) { + this->status = val; +} + +void TProgressUpdateResp::__set_footerSummary(const std::string& val) { + this->footerSummary = val; +} + +void TProgressUpdateResp::__set_startTime(const int64_t val) { + this->startTime = val; +} + +uint32_t TProgressUpdateResp::read(::apache::thrift::protocol::TProtocol* iprot) { + + apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + bool isset_headerNames = false; + bool isset_rows = false; + bool isset_progressedPercentage = false; + bool isset_status = false; + bool isset_footerSummary = false; + bool isset_startTime = false; + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + case 1: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->headerNames.clear(); + uint32_t _size304; + ::apache::thrift::protocol::TType _etype307; + xfer += iprot->readListBegin(_etype307, _size304); + this->headerNames.resize(_size304); + uint32_t _i308; + for (_i308 = 0; _i308 < _size304; ++_i308) + { + xfer += iprot->readString(this->headerNames[_i308]); + } + xfer += iprot->readListEnd(); + } + isset_headerNames = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->rows.clear(); + uint32_t _size309; + ::apache::thrift::protocol::TType _etype312; + xfer += iprot->readListBegin(_etype312, _size309); + this->rows.resize(_size309); + uint32_t _i313; + for (_i313 = 0; _i313 < _size309; ++_i313) + { + { + this->rows[_i313].clear(); + uint32_t _size314; + ::apache::thrift::protocol::TType _etype317; + xfer += iprot->readListBegin(_etype317, _size314); + this->rows[_i313].resize(_size314); + uint32_t _i318; + for (_i318 = 0; _i318 < _size314; ++_i318) + { + xfer += iprot->readString(this->rows[_i313][_i318]); + } + xfer += iprot->readListEnd(); + } + } + xfer += iprot->readListEnd(); + } + isset_rows = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: + if (ftype == ::apache::thrift::protocol::T_DOUBLE) { + xfer += iprot->readDouble(this->progressedPercentage); + isset_progressedPercentage = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast319; + xfer += iprot->readI32(ecast319); + this->status = (TJobExecutionStatus::type)ecast319; + isset_status = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->footerSummary); + isset_footerSummary = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->startTime); + isset_startTime = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_headerNames) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_rows) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_progressedPercentage) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_status) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_footerSummary) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_startTime) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t TProgressUpdateResp::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); + xfer += oprot->writeStructBegin("TProgressUpdateResp"); + + xfer += oprot->writeFieldBegin("headerNames", ::apache::thrift::protocol::T_LIST, 1); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->headerNames.size())); + std::vector ::const_iterator _iter320; + for (_iter320 = this->headerNames.begin(); _iter320 != this->headerNames.end(); ++_iter320) + { + xfer += oprot->writeString((*_iter320)); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("rows", ::apache::thrift::protocol::T_LIST, 2); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_LIST, static_cast(this->rows.size())); + std::vector > ::const_iterator _iter321; + for (_iter321 = this->rows.begin(); _iter321 != this->rows.end(); ++_iter321) + { + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast((*_iter321).size())); + std::vector ::const_iterator _iter322; + for (_iter322 = (*_iter321).begin(); _iter322 != (*_iter321).end(); ++_iter322) + { + xfer += oprot->writeString((*_iter322)); + } + xfer += oprot->writeListEnd(); + } + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("progressedPercentage", ::apache::thrift::protocol::T_DOUBLE, 3); + xfer += oprot->writeDouble(this->progressedPercentage); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("status", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32((int32_t)this->status); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("footerSummary", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->footerSummary); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("startTime", ::apache::thrift::protocol::T_I64, 6); + xfer += oprot->writeI64(this->startTime); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TProgressUpdateResp &a, TProgressUpdateResp &b) { + using ::std::swap; + swap(a.headerNames, b.headerNames); + swap(a.rows, b.rows); + swap(a.progressedPercentage, b.progressedPercentage); + swap(a.status, b.status); + swap(a.footerSummary, b.footerSummary); + swap(a.startTime, b.startTime); +} + +TProgressUpdateResp::TProgressUpdateResp(const TProgressUpdateResp& other323) { + headerNames = other323.headerNames; + rows = other323.rows; + progressedPercentage = other323.progressedPercentage; + status = other323.status; + footerSummary = other323.footerSummary; + startTime = other323.startTime; +} +TProgressUpdateResp& TProgressUpdateResp::operator=(const TProgressUpdateResp& other324) { + headerNames = other324.headerNames; + rows = other324.rows; + progressedPercentage = other324.progressedPercentage; + status = other324.status; + footerSummary = other324.footerSummary; + startTime = other324.startTime; + return *this; +} +void TProgressUpdateResp::printTo(std::ostream& out) const { + using ::apache::thrift::to_string; + out << "TProgressUpdateResp("; + out << "headerNames=" << to_string(headerNames); + out << ", " << "rows=" << to_string(rows); + out << ", " << "progressedPercentage=" << to_string(progressedPercentage); + out << ", " << "status=" << to_string(status); + out << ", " << "footerSummary=" << to_string(footerSummary); + out << ", " << "startTime=" << to_string(startTime); + out << ")"; +} + }}}}} // namespace diff --git service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h index b249544..94adda3 100644 --- service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h +++ service-rpc/src/gen/thrift/gen-cpp/TCLIService_types.h @@ -175,6 +175,21 @@ struct TFetchOrientation { extern const std::map _TFetchOrientation_VALUES_TO_NAMES; +struct TJobExecutionStatus { + enum type { + SUBMITTED = 0, + INITING = 1, + RUNNING = 2, + SUCCEEDED = 3, + KILLED = 4, + FAILED = 5, + ERROR = 6, + NOT_AVAILABLE = 7 + }; +}; + +extern const std::map _TJobExecutionStatus_VALUES_TO_NAMES; + typedef int32_t TTypeEntryPtr; typedef std::string TIdentifier; @@ -339,6 +354,10 @@ class TRenewDelegationTokenReq; class TRenewDelegationTokenResp; +class TProgressUpdateReq; + +class TProgressUpdateResp; + typedef struct _TTypeQualifierValue__isset { _TTypeQualifierValue__isset() : i32Value(false), stringValue(false) {} bool i32Value :1; @@ -4470,6 +4489,111 @@ inline std::ostream& operator<<(std::ostream& out, const TRenewDelegationTokenRe return out; } + +class TProgressUpdateReq { + public: + + TProgressUpdateReq(const TProgressUpdateReq&); + TProgressUpdateReq& operator=(const TProgressUpdateReq&); + TProgressUpdateReq() { + } + + virtual ~TProgressUpdateReq() throw(); + TOperationHandle operationHandle; + + void __set_operationHandle(const TOperationHandle& val); + + bool operator == (const TProgressUpdateReq & rhs) const + { + if (!(operationHandle == rhs.operationHandle)) + return false; + return true; + } + bool operator != (const TProgressUpdateReq &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TProgressUpdateReq & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + + virtual void printTo(std::ostream& out) const; +}; + +void swap(TProgressUpdateReq &a, TProgressUpdateReq &b); + +inline std::ostream& operator<<(std::ostream& out, const TProgressUpdateReq& obj) +{ + obj.printTo(out); + return out; +} + + +class TProgressUpdateResp { + public: + + TProgressUpdateResp(const TProgressUpdateResp&); + TProgressUpdateResp& operator=(const TProgressUpdateResp&); + TProgressUpdateResp() : progressedPercentage(0), status((TJobExecutionStatus::type)0), footerSummary(), startTime(0) { + } + + virtual ~TProgressUpdateResp() throw(); + std::vector headerNames; + std::vector > rows; + double progressedPercentage; + TJobExecutionStatus::type status; + std::string footerSummary; + int64_t startTime; + + void __set_headerNames(const std::vector & val); + + void __set_rows(const std::vector > & val); + + void __set_progressedPercentage(const double val); + + void __set_status(const TJobExecutionStatus::type val); + + void __set_footerSummary(const std::string& val); + + void __set_startTime(const int64_t val); + + bool operator == (const TProgressUpdateResp & rhs) const + { + if (!(headerNames == rhs.headerNames)) + return false; + if (!(rows == rhs.rows)) + return false; + if (!(progressedPercentage == rhs.progressedPercentage)) + return false; + if (!(status == rhs.status)) + return false; + if (!(footerSummary == rhs.footerSummary)) + return false; + if (!(startTime == rhs.startTime)) + return false; + return true; + } + bool operator != (const TProgressUpdateResp &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TProgressUpdateResp & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + + virtual void printTo(std::ostream& out) const; +}; + +void swap(TProgressUpdateResp &a, TProgressUpdateResp &b); + +inline std::ostream& operator<<(std::ostream& out, const TProgressUpdateResp& obj) +{ + obj.printTo(out); + return out; +} + }}}}} // namespace #endif diff --git service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIService.java service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIService.java index 6dba051..c144d5b 100644 --- service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIService.java +++ service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TCLIService.java @@ -6,33 +6,20 @@ */ package org.apache.hive.service.rpc.thrift; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.protocol.TTupleProtocol; import org.apache.thrift.scheme.IScheme; import org.apache.thrift.scheme.SchemeFactory; import org.apache.thrift.scheme.StandardScheme; - import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import org.apache.thrift.async.AsyncMethodCallback; -import org.apache.thrift.server.AbstractNonblockingServer.*; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import javax.annotation.Generated; +import org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Generated; +import java.util.*; + @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) @Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") public class TCLIService { @@ -81,6 +68,8 @@ public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException; + public TProgressUpdateResp GetProgressUpdate(TProgressUpdateReq req) throws org.apache.thrift.TException; + } public interface AsyncIface { @@ -127,6 +116,8 @@ public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void GetProgressUpdate(TProgressUpdateReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + } public static class Client extends org.apache.thrift.TServiceClient implements Iface { @@ -632,6 +623,29 @@ public TRenewDelegationTokenResp recv_RenewDelegationToken() throws org.apache.t throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "RenewDelegationToken failed: unknown result"); } + public TProgressUpdateResp GetProgressUpdate(TProgressUpdateReq req) throws org.apache.thrift.TException + { + send_GetProgressUpdate(req); + return recv_GetProgressUpdate(); + } + + public void send_GetProgressUpdate(TProgressUpdateReq req) throws org.apache.thrift.TException + { + GetProgressUpdate_args args = new GetProgressUpdate_args(); + args.setReq(req); + sendBase("GetProgressUpdate", args); + } + + public TProgressUpdateResp recv_GetProgressUpdate() throws org.apache.thrift.TException + { + GetProgressUpdate_result result = new GetProgressUpdate_result(); + receiveBase(result, "GetProgressUpdate"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetProgressUpdate failed: unknown result"); + } + } public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -1322,6 +1336,38 @@ public TRenewDelegationTokenResp getResult() throws org.apache.thrift.TException } } + public void GetProgressUpdate(TProgressUpdateReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetProgressUpdate_call method_call = new GetProgressUpdate_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetProgressUpdate_call extends org.apache.thrift.async.TAsyncMethodCall { + private TProgressUpdateReq req; + public GetProgressUpdate_call(TProgressUpdateReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetProgressUpdate", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetProgressUpdate_args args = new GetProgressUpdate_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TProgressUpdateResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetProgressUpdate(); + } + } + } public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { @@ -1356,6 +1402,7 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { + public GetProgressUpdate() { + super("GetProgressUpdate"); + } + + public GetProgressUpdate_args getEmptyArgsInstance() { + return new GetProgressUpdate_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetProgressUpdate_result getResult(I iface, GetProgressUpdate_args args) throws org.apache.thrift.TException { + GetProgressUpdate_result result = new GetProgressUpdate_result(); + result.success = iface.GetProgressUpdate(args.req); + return result; + } + } + } public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { @@ -1813,6 +1880,7 @@ protected AsyncProcessor(I iface, Map extends org.apache.thrift.AsyncProcessFunction { + public GetProgressUpdate() { + super("GetProgressUpdate"); + } + + public GetProgressUpdate_args getEmptyArgsInstance() { + return new GetProgressUpdate_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TProgressUpdateResp o) { + GetProgressUpdate_result result = new GetProgressUpdate_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetProgressUpdate_result result = new GetProgressUpdate_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetProgressUpdate_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetProgressUpdate(args.req,resultHandler); + } + } + } public static class OpenSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { @@ -18135,4 +18254,730 @@ public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken } + public static class GetProgressUpdate_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetProgressUpdate_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetProgressUpdate_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetProgressUpdate_argsTupleSchemeFactory()); + } + + private TProgressUpdateReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TProgressUpdateReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetProgressUpdate_args.class, metaDataMap); + } + + public GetProgressUpdate_args() { + } + + public GetProgressUpdate_args( + TProgressUpdateReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetProgressUpdate_args(GetProgressUpdate_args other) { + if (other.isSetReq()) { + this.req = new TProgressUpdateReq(other.req); + } + } + + public GetProgressUpdate_args deepCopy() { + return new GetProgressUpdate_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TProgressUpdateReq getReq() { + return this.req; + } + + public void setReq(TProgressUpdateReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TProgressUpdateReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetProgressUpdate_args) + return this.equals((GetProgressUpdate_args)that); + return false; + } + + public boolean equals(GetProgressUpdate_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetProgressUpdate_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetProgressUpdate_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetProgressUpdate_argsStandardSchemeFactory implements SchemeFactory { + public GetProgressUpdate_argsStandardScheme getScheme() { + return new GetProgressUpdate_argsStandardScheme(); + } + } + + private static class GetProgressUpdate_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetProgressUpdate_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TProgressUpdateReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetProgressUpdate_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetProgressUpdate_argsTupleSchemeFactory implements SchemeFactory { + public GetProgressUpdate_argsTupleScheme getScheme() { + return new GetProgressUpdate_argsTupleScheme(); + } + } + + private static class GetProgressUpdate_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetProgressUpdate_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetProgressUpdate_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TProgressUpdateReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetProgressUpdate_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetProgressUpdate_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetProgressUpdate_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetProgressUpdate_resultTupleSchemeFactory()); + } + + private TProgressUpdateResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TProgressUpdateResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetProgressUpdate_result.class, metaDataMap); + } + + public GetProgressUpdate_result() { + } + + public GetProgressUpdate_result( + TProgressUpdateResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetProgressUpdate_result(GetProgressUpdate_result other) { + if (other.isSetSuccess()) { + this.success = new TProgressUpdateResp(other.success); + } + } + + public GetProgressUpdate_result deepCopy() { + return new GetProgressUpdate_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TProgressUpdateResp getSuccess() { + return this.success; + } + + public void setSuccess(TProgressUpdateResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TProgressUpdateResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetProgressUpdate_result) + return this.equals((GetProgressUpdate_result)that); + return false; + } + + public boolean equals(GetProgressUpdate_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetProgressUpdate_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetProgressUpdate_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetProgressUpdate_resultStandardSchemeFactory implements SchemeFactory { + public GetProgressUpdate_resultStandardScheme getScheme() { + return new GetProgressUpdate_resultStandardScheme(); + } + } + + private static class GetProgressUpdate_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetProgressUpdate_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TProgressUpdateResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetProgressUpdate_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetProgressUpdate_resultTupleSchemeFactory implements SchemeFactory { + public GetProgressUpdate_resultTupleScheme getScheme() { + return new GetProgressUpdate_resultTupleScheme(); + } + } + + private static class GetProgressUpdate_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetProgressUpdate_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetProgressUpdate_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TProgressUpdateResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + } diff --git service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java new file mode 100644 index 0000000..05d6ba1 --- /dev/null +++ service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java @@ -0,0 +1,59 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +public enum TJobExecutionStatus implements org.apache.thrift.TEnum { + SUBMITTED(0), + INITING(1), + RUNNING(2), + SUCCEEDED(3), + KILLED(4), + FAILED(5), + ERROR(6), + NOT_AVAILABLE(7); + + private final int value; + + private TJobExecutionStatus(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TJobExecutionStatus findByValue(int value) { + switch (value) { + case 0: + return SUBMITTED; + case 1: + return INITING; + case 2: + return RUNNING; + case 3: + return SUCCEEDED; + case 4: + return KILLED; + case 5: + return FAILED; + case 6: + return ERROR; + case 7: + return NOT_AVAILABLE; + default: + return null; + } + } +} diff --git service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TProgressUpdateReq.java service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TProgressUpdateReq.java new file mode 100644 index 0000000..8846f7b --- /dev/null +++ service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TProgressUpdateReq.java @@ -0,0 +1,376 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; +import org.apache.thrift.scheme.TupleScheme; + +import javax.annotation.Generated; +import java.util.*; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TProgressUpdateReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TProgressUpdateReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TProgressUpdateReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TProgressUpdateReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TProgressUpdateReq.class, metaDataMap); + } + + public TProgressUpdateReq() { + } + + public TProgressUpdateReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TProgressUpdateReq(TProgressUpdateReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TProgressUpdateReq deepCopy() { + return new TProgressUpdateReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TProgressUpdateReq) + return this.equals((TProgressUpdateReq)that); + return false; + } + + public boolean equals(TProgressUpdateReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TProgressUpdateReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TProgressUpdateReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TProgressUpdateReqStandardSchemeFactory implements SchemeFactory { + public TProgressUpdateReqStandardScheme getScheme() { + return new TProgressUpdateReqStandardScheme(); + } + } + + private static class TProgressUpdateReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TProgressUpdateReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TProgressUpdateReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TProgressUpdateReqTupleSchemeFactory implements SchemeFactory { + public TProgressUpdateReqTupleScheme getScheme() { + return new TProgressUpdateReqTupleScheme(); + } + } + + private static class TProgressUpdateReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TProgressUpdateReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TProgressUpdateReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java new file mode 100644 index 0000000..dd8a4ac --- /dev/null +++ service-rpc/src/gen/thrift/gen-javabean/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java @@ -0,0 +1,1016 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; +import org.apache.thrift.scheme.TupleScheme; + +import javax.annotation.Generated; +import java.util.*; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TProgressUpdateResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TProgressUpdateResp"); + + private static final org.apache.thrift.protocol.TField HEADER_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("headerNames", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("rows", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField PROGRESSED_PERCENTAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("progressedPercentage", org.apache.thrift.protocol.TType.DOUBLE, (short)3); + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField FOOTER_SUMMARY_FIELD_DESC = new org.apache.thrift.protocol.TField("footerSummary", org.apache.thrift.protocol.TType.STRING, (short)5); + private static final org.apache.thrift.protocol.TField START_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("startTime", org.apache.thrift.protocol.TType.I64, (short)6); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TProgressUpdateRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TProgressUpdateRespTupleSchemeFactory()); + } + + private List headerNames; // required + private List> rows; // required + private double progressedPercentage; // required + private TJobExecutionStatus status; // required + private String footerSummary; // required + private long startTime; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + HEADER_NAMES((short)1, "headerNames"), + ROWS((short)2, "rows"), + PROGRESSED_PERCENTAGE((short)3, "progressedPercentage"), + /** + * + * @see TJobExecutionStatus + */ + STATUS((short)4, "status"), + FOOTER_SUMMARY((short)5, "footerSummary"), + START_TIME((short)6, "startTime"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // HEADER_NAMES + return HEADER_NAMES; + case 2: // ROWS + return ROWS; + case 3: // PROGRESSED_PERCENTAGE + return PROGRESSED_PERCENTAGE; + case 4: // STATUS + return STATUS; + case 5: // FOOTER_SUMMARY + return FOOTER_SUMMARY; + case 6: // START_TIME + return START_TIME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __PROGRESSEDPERCENTAGE_ISSET_ID = 0; + private static final int __STARTTIME_ISSET_ID = 1; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.HEADER_NAMES, new org.apache.thrift.meta_data.FieldMetaData("headerNames", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.ROWS, new org.apache.thrift.meta_data.FieldMetaData("rows", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))))); + tmpMap.put(_Fields.PROGRESSED_PERCENTAGE, new org.apache.thrift.meta_data.FieldMetaData("progressedPercentage", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TJobExecutionStatus.class))); + tmpMap.put(_Fields.FOOTER_SUMMARY, new org.apache.thrift.meta_data.FieldMetaData("footerSummary", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.START_TIME, new org.apache.thrift.meta_data.FieldMetaData("startTime", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TProgressUpdateResp.class, metaDataMap); + } + + public TProgressUpdateResp() { + } + + public TProgressUpdateResp( + List headerNames, + List> rows, + double progressedPercentage, + TJobExecutionStatus status, + String footerSummary, + long startTime) + { + this(); + this.headerNames = headerNames; + this.rows = rows; + this.progressedPercentage = progressedPercentage; + setProgressedPercentageIsSet(true); + this.status = status; + this.footerSummary = footerSummary; + this.startTime = startTime; + setStartTimeIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TProgressUpdateResp(TProgressUpdateResp other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetHeaderNames()) { + List __this__headerNames = new ArrayList(other.headerNames); + this.headerNames = __this__headerNames; + } + if (other.isSetRows()) { + List> __this__rows = new ArrayList>(other.rows.size()); + for (List other_element : other.rows) { + List __this__rows_copy = new ArrayList(other_element); + __this__rows.add(__this__rows_copy); + } + this.rows = __this__rows; + } + this.progressedPercentage = other.progressedPercentage; + if (other.isSetStatus()) { + this.status = other.status; + } + if (other.isSetFooterSummary()) { + this.footerSummary = other.footerSummary; + } + this.startTime = other.startTime; + } + + public TProgressUpdateResp deepCopy() { + return new TProgressUpdateResp(this); + } + + @Override + public void clear() { + this.headerNames = null; + this.rows = null; + setProgressedPercentageIsSet(false); + this.progressedPercentage = 0.0; + this.status = null; + this.footerSummary = null; + setStartTimeIsSet(false); + this.startTime = 0; + } + + public int getHeaderNamesSize() { + return (this.headerNames == null) ? 0 : this.headerNames.size(); + } + + public java.util.Iterator getHeaderNamesIterator() { + return (this.headerNames == null) ? null : this.headerNames.iterator(); + } + + public void addToHeaderNames(String elem) { + if (this.headerNames == null) { + this.headerNames = new ArrayList(); + } + this.headerNames.add(elem); + } + + public List getHeaderNames() { + return this.headerNames; + } + + public void setHeaderNames(List headerNames) { + this.headerNames = headerNames; + } + + public void unsetHeaderNames() { + this.headerNames = null; + } + + /** Returns true if field headerNames is set (has been assigned a value) and false otherwise */ + public boolean isSetHeaderNames() { + return this.headerNames != null; + } + + public void setHeaderNamesIsSet(boolean value) { + if (!value) { + this.headerNames = null; + } + } + + public int getRowsSize() { + return (this.rows == null) ? 0 : this.rows.size(); + } + + public java.util.Iterator> getRowsIterator() { + return (this.rows == null) ? null : this.rows.iterator(); + } + + public void addToRows(List elem) { + if (this.rows == null) { + this.rows = new ArrayList>(); + } + this.rows.add(elem); + } + + public List> getRows() { + return this.rows; + } + + public void setRows(List> rows) { + this.rows = rows; + } + + public void unsetRows() { + this.rows = null; + } + + /** Returns true if field rows is set (has been assigned a value) and false otherwise */ + public boolean isSetRows() { + return this.rows != null; + } + + public void setRowsIsSet(boolean value) { + if (!value) { + this.rows = null; + } + } + + public double getProgressedPercentage() { + return this.progressedPercentage; + } + + public void setProgressedPercentage(double progressedPercentage) { + this.progressedPercentage = progressedPercentage; + setProgressedPercentageIsSet(true); + } + + public void unsetProgressedPercentage() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROGRESSEDPERCENTAGE_ISSET_ID); + } + + /** Returns true if field progressedPercentage is set (has been assigned a value) and false otherwise */ + public boolean isSetProgressedPercentage() { + return EncodingUtils.testBit(__isset_bitfield, __PROGRESSEDPERCENTAGE_ISSET_ID); + } + + public void setProgressedPercentageIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROGRESSEDPERCENTAGE_ISSET_ID, value); + } + + /** + * + * @see TJobExecutionStatus + */ + public TJobExecutionStatus getStatus() { + return this.status; + } + + /** + * + * @see TJobExecutionStatus + */ + public void setStatus(TJobExecutionStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public String getFooterSummary() { + return this.footerSummary; + } + + public void setFooterSummary(String footerSummary) { + this.footerSummary = footerSummary; + } + + public void unsetFooterSummary() { + this.footerSummary = null; + } + + /** Returns true if field footerSummary is set (has been assigned a value) and false otherwise */ + public boolean isSetFooterSummary() { + return this.footerSummary != null; + } + + public void setFooterSummaryIsSet(boolean value) { + if (!value) { + this.footerSummary = null; + } + } + + public long getStartTime() { + return this.startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + setStartTimeIsSet(true); + } + + public void unsetStartTime() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STARTTIME_ISSET_ID); + } + + /** Returns true if field startTime is set (has been assigned a value) and false otherwise */ + public boolean isSetStartTime() { + return EncodingUtils.testBit(__isset_bitfield, __STARTTIME_ISSET_ID); + } + + public void setStartTimeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STARTTIME_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case HEADER_NAMES: + if (value == null) { + unsetHeaderNames(); + } else { + setHeaderNames((List)value); + } + break; + + case ROWS: + if (value == null) { + unsetRows(); + } else { + setRows((List>)value); + } + break; + + case PROGRESSED_PERCENTAGE: + if (value == null) { + unsetProgressedPercentage(); + } else { + setProgressedPercentage((Double)value); + } + break; + + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TJobExecutionStatus)value); + } + break; + + case FOOTER_SUMMARY: + if (value == null) { + unsetFooterSummary(); + } else { + setFooterSummary((String)value); + } + break; + + case START_TIME: + if (value == null) { + unsetStartTime(); + } else { + setStartTime((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case HEADER_NAMES: + return getHeaderNames(); + + case ROWS: + return getRows(); + + case PROGRESSED_PERCENTAGE: + return getProgressedPercentage(); + + case STATUS: + return getStatus(); + + case FOOTER_SUMMARY: + return getFooterSummary(); + + case START_TIME: + return getStartTime(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case HEADER_NAMES: + return isSetHeaderNames(); + case ROWS: + return isSetRows(); + case PROGRESSED_PERCENTAGE: + return isSetProgressedPercentage(); + case STATUS: + return isSetStatus(); + case FOOTER_SUMMARY: + return isSetFooterSummary(); + case START_TIME: + return isSetStartTime(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TProgressUpdateResp) + return this.equals((TProgressUpdateResp)that); + return false; + } + + public boolean equals(TProgressUpdateResp that) { + if (that == null) + return false; + + boolean this_present_headerNames = true && this.isSetHeaderNames(); + boolean that_present_headerNames = true && that.isSetHeaderNames(); + if (this_present_headerNames || that_present_headerNames) { + if (!(this_present_headerNames && that_present_headerNames)) + return false; + if (!this.headerNames.equals(that.headerNames)) + return false; + } + + boolean this_present_rows = true && this.isSetRows(); + boolean that_present_rows = true && that.isSetRows(); + if (this_present_rows || that_present_rows) { + if (!(this_present_rows && that_present_rows)) + return false; + if (!this.rows.equals(that.rows)) + return false; + } + + boolean this_present_progressedPercentage = true; + boolean that_present_progressedPercentage = true; + if (this_present_progressedPercentage || that_present_progressedPercentage) { + if (!(this_present_progressedPercentage && that_present_progressedPercentage)) + return false; + if (this.progressedPercentage != that.progressedPercentage) + return false; + } + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_footerSummary = true && this.isSetFooterSummary(); + boolean that_present_footerSummary = true && that.isSetFooterSummary(); + if (this_present_footerSummary || that_present_footerSummary) { + if (!(this_present_footerSummary && that_present_footerSummary)) + return false; + if (!this.footerSummary.equals(that.footerSummary)) + return false; + } + + boolean this_present_startTime = true; + boolean that_present_startTime = true; + if (this_present_startTime || that_present_startTime) { + if (!(this_present_startTime && that_present_startTime)) + return false; + if (this.startTime != that.startTime) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_headerNames = true && (isSetHeaderNames()); + list.add(present_headerNames); + if (present_headerNames) + list.add(headerNames); + + boolean present_rows = true && (isSetRows()); + list.add(present_rows); + if (present_rows) + list.add(rows); + + boolean present_progressedPercentage = true; + list.add(present_progressedPercentage); + if (present_progressedPercentage) + list.add(progressedPercentage); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status.getValue()); + + boolean present_footerSummary = true && (isSetFooterSummary()); + list.add(present_footerSummary); + if (present_footerSummary) + list.add(footerSummary); + + boolean present_startTime = true; + list.add(present_startTime); + if (present_startTime) + list.add(startTime); + + return list.hashCode(); + } + + @Override + public int compareTo(TProgressUpdateResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetHeaderNames()).compareTo(other.isSetHeaderNames()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHeaderNames()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.headerNames, other.headerNames); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRows()).compareTo(other.isSetRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, other.rows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetProgressedPercentage()).compareTo(other.isSetProgressedPercentage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetProgressedPercentage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.progressedPercentage, other.progressedPercentage); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFooterSummary()).compareTo(other.isSetFooterSummary()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFooterSummary()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.footerSummary, other.footerSummary); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStartTime()).compareTo(other.isSetStartTime()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStartTime()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startTime, other.startTime); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TProgressUpdateResp("); + boolean first = true; + + sb.append("headerNames:"); + if (this.headerNames == null) { + sb.append("null"); + } else { + sb.append(this.headerNames); + } + first = false; + if (!first) sb.append(", "); + sb.append("rows:"); + if (this.rows == null) { + sb.append("null"); + } else { + sb.append(this.rows); + } + first = false; + if (!first) sb.append(", "); + sb.append("progressedPercentage:"); + sb.append(this.progressedPercentage); + first = false; + if (!first) sb.append(", "); + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("footerSummary:"); + if (this.footerSummary == null) { + sb.append("null"); + } else { + sb.append(this.footerSummary); + } + first = false; + if (!first) sb.append(", "); + sb.append("startTime:"); + sb.append(this.startTime); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetHeaderNames()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'headerNames' is unset! Struct:" + toString()); + } + + if (!isSetRows()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'rows' is unset! Struct:" + toString()); + } + + if (!isSetProgressedPercentage()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'progressedPercentage' is unset! Struct:" + toString()); + } + + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!isSetFooterSummary()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'footerSummary' is unset! Struct:" + toString()); + } + + if (!isSetStartTime()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'startTime' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TProgressUpdateRespStandardSchemeFactory implements SchemeFactory { + public TProgressUpdateRespStandardScheme getScheme() { + return new TProgressUpdateRespStandardScheme(); + } + } + + private static class TProgressUpdateRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // HEADER_NAMES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list180 = iprot.readListBegin(); + struct.headerNames = new ArrayList(_list180.size); + String _elem181; + for (int _i182 = 0; _i182 < _list180.size; ++_i182) + { + _elem181 = iprot.readString(); + struct.headerNames.add(_elem181); + } + iprot.readListEnd(); + } + struct.setHeaderNamesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list183 = iprot.readListBegin(); + struct.rows = new ArrayList>(_list183.size); + List _elem184; + for (int _i185 = 0; _i185 < _list183.size; ++_i185) + { + { + org.apache.thrift.protocol.TList _list186 = iprot.readListBegin(); + _elem184 = new ArrayList(_list186.size); + String _elem187; + for (int _i188 = 0; _i188 < _list186.size; ++_i188) + { + _elem187 = iprot.readString(); + _elem184.add(_elem187); + } + iprot.readListEnd(); + } + struct.rows.add(_elem184); + } + iprot.readListEnd(); + } + struct.setRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PROGRESSED_PERCENTAGE + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.progressedPercentage = iprot.readDouble(); + struct.setProgressedPercentageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.status = org.apache.hive.service.rpc.thrift.TJobExecutionStatus.findByValue(iprot.readI32()); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // FOOTER_SUMMARY + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.footerSummary = iprot.readString(); + struct.setFooterSummaryIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // START_TIME + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.startTime = iprot.readI64(); + struct.setStartTimeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.headerNames != null) { + oprot.writeFieldBegin(HEADER_NAMES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.headerNames.size())); + for (String _iter189 : struct.headerNames) + { + oprot.writeString(_iter189); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.rows != null) { + oprot.writeFieldBegin(ROWS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.LIST, struct.rows.size())); + for (List _iter190 : struct.rows) + { + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter190.size())); + for (String _iter191 : _iter190) + { + oprot.writeString(_iter191); + } + oprot.writeListEnd(); + } + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(PROGRESSED_PERCENTAGE_FIELD_DESC); + oprot.writeDouble(struct.progressedPercentage); + oprot.writeFieldEnd(); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + oprot.writeI32(struct.status.getValue()); + oprot.writeFieldEnd(); + } + if (struct.footerSummary != null) { + oprot.writeFieldBegin(FOOTER_SUMMARY_FIELD_DESC); + oprot.writeString(struct.footerSummary); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(START_TIME_FIELD_DESC); + oprot.writeI64(struct.startTime); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TProgressUpdateRespTupleSchemeFactory implements SchemeFactory { + public TProgressUpdateRespTupleScheme getScheme() { + return new TProgressUpdateRespTupleScheme(); + } + } + + private static class TProgressUpdateRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.headerNames.size()); + for (String _iter192 : struct.headerNames) + { + oprot.writeString(_iter192); + } + } + { + oprot.writeI32(struct.rows.size()); + for (List _iter193 : struct.rows) + { + { + oprot.writeI32(_iter193.size()); + for (String _iter194 : _iter193) + { + oprot.writeString(_iter194); + } + } + } + } + oprot.writeDouble(struct.progressedPercentage); + oprot.writeI32(struct.status.getValue()); + oprot.writeString(struct.footerSummary); + oprot.writeI64(struct.startTime); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list195 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.headerNames = new ArrayList(_list195.size); + String _elem196; + for (int _i197 = 0; _i197 < _list195.size; ++_i197) + { + _elem196 = iprot.readString(); + struct.headerNames.add(_elem196); + } + } + struct.setHeaderNamesIsSet(true); + { + org.apache.thrift.protocol.TList _list198 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.LIST, iprot.readI32()); + struct.rows = new ArrayList>(_list198.size); + List _elem199; + for (int _i200 = 0; _i200 < _list198.size; ++_i200) + { + { + org.apache.thrift.protocol.TList _list201 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + _elem199 = new ArrayList(_list201.size); + String _elem202; + for (int _i203 = 0; _i203 < _list201.size; ++_i203) + { + _elem202 = iprot.readString(); + _elem199.add(_elem202); + } + } + struct.rows.add(_elem199); + } + } + struct.setRowsIsSet(true); + struct.progressedPercentage = iprot.readDouble(); + struct.setProgressedPercentageIsSet(true); + struct.status = org.apache.hive.service.rpc.thrift.TJobExecutionStatus.findByValue(iprot.readI32()); + struct.setStatusIsSet(true); + struct.footerSummary = iprot.readString(); + struct.setFooterSummaryIsSet(true); + struct.startTime = iprot.readI64(); + struct.setStartTimeIsSet(true); + } + } + +} + diff --git service-rpc/src/gen/thrift/gen-php/TCLIService.php service-rpc/src/gen/thrift/gen-php/TCLIService.php index d283145..f6ae2e3 100644 --- service-rpc/src/gen/thrift/gen-php/TCLIService.php +++ service-rpc/src/gen/thrift/gen-php/TCLIService.php @@ -121,6 +121,11 @@ interface TCLIServiceIf { * @return \TRenewDelegationTokenResp */ public function RenewDelegationToken(\TRenewDelegationTokenReq $req); + /** + * @param \TProgressUpdateReq $req + * @return \TProgressUpdateResp + */ + public function GetProgressUpdate(\TProgressUpdateReq $req); } class TCLIServiceClient implements \TCLIServiceIf { @@ -1205,6 +1210,57 @@ class TCLIServiceClient implements \TCLIServiceIf { throw new \Exception("RenewDelegationToken failed: unknown result"); } + public function GetProgressUpdate(\TProgressUpdateReq $req) + { + $this->send_GetProgressUpdate($req); + return $this->recv_GetProgressUpdate(); + } + + public function send_GetProgressUpdate(\TProgressUpdateReq $req) + { + $args = new \TCLIService_GetProgressUpdate_args(); + $args->req = $req; + $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'GetProgressUpdate', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('GetProgressUpdate', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_GetProgressUpdate() + { + $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\TCLIService_GetProgressUpdate_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \TCLIService_GetProgressUpdate_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + throw new \Exception("GetProgressUpdate failed: unknown result"); + } + } // HELPER FUNCTIONS AND STRUCTURES @@ -4569,4 +4625,164 @@ class TCLIService_RenewDelegationToken_result { } +class TCLIService_GetProgressUpdate_args { + static $_TSPEC; + + /** + * @var \TProgressUpdateReq + */ + public $req = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'req', + 'type' => TType::STRUCT, + 'class' => '\TProgressUpdateReq', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['req'])) { + $this->req = $vals['req']; + } + } + } + + public function getName() { + return 'TCLIService_GetProgressUpdate_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->req = new \TProgressUpdateReq(); + $xfer += $this->req->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('TCLIService_GetProgressUpdate_args'); + if ($this->req !== null) { + if (!is_object($this->req)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('req', TType::STRUCT, 1); + $xfer += $this->req->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class TCLIService_GetProgressUpdate_result { + static $_TSPEC; + + /** + * @var \TProgressUpdateResp + */ + public $success = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\TProgressUpdateResp', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + } + } + + public function getName() { + return 'TCLIService_GetProgressUpdate_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \TProgressUpdateResp(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('TCLIService_GetProgressUpdate_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + diff --git service-rpc/src/gen/thrift/gen-php/Types.php service-rpc/src/gen/thrift/gen-php/Types.php index 786c773..952e6be 100644 --- service-rpc/src/gen/thrift/gen-php/Types.php +++ service-rpc/src/gen/thrift/gen-php/Types.php @@ -264,6 +264,27 @@ final class TFetchOrientation { ); } +final class TJobExecutionStatus { + const SUBMITTED = 0; + const INITING = 1; + const RUNNING = 2; + const SUCCEEDED = 3; + const KILLED = 4; + const FAILED = 5; + const ERROR = 6; + const NOT_AVAILABLE = 7; + static public $__names = array( + 0 => 'SUBMITTED', + 1 => 'INITING', + 2 => 'RUNNING', + 3 => 'SUCCEEDED', + 4 => 'KILLED', + 5 => 'FAILED', + 6 => 'ERROR', + 7 => 'NOT_AVAILABLE', + ); +} + class TTypeQualifierValue { static $_TSPEC; @@ -9696,6 +9717,351 @@ class TRenewDelegationTokenResp { } +class TProgressUpdateReq { + static $_TSPEC; + + /** + * @var \TOperationHandle + */ + public $operationHandle = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'operationHandle', + 'type' => TType::STRUCT, + 'class' => '\TOperationHandle', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['operationHandle'])) { + $this->operationHandle = $vals['operationHandle']; + } + } + } + + public function getName() { + return 'TProgressUpdateReq'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->operationHandle = new \TOperationHandle(); + $xfer += $this->operationHandle->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('TProgressUpdateReq'); + if ($this->operationHandle !== null) { + if (!is_object($this->operationHandle)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('operationHandle', TType::STRUCT, 1); + $xfer += $this->operationHandle->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class TProgressUpdateResp { + static $_TSPEC; + + /** + * @var string[] + */ + public $headerNames = null; + /** + * @var (string[])[] + */ + public $rows = null; + /** + * @var double + */ + public $progressedPercentage = null; + /** + * @var int + */ + public $status = null; + /** + * @var string + */ + public $footerSummary = null; + /** + * @var int + */ + public $startTime = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'headerNames', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 2 => array( + 'var' => 'rows', + 'type' => TType::LST, + 'etype' => TType::LST, + 'elem' => array( + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + ), + 3 => array( + 'var' => 'progressedPercentage', + 'type' => TType::DOUBLE, + ), + 4 => array( + 'var' => 'status', + 'type' => TType::I32, + ), + 5 => array( + 'var' => 'footerSummary', + 'type' => TType::STRING, + ), + 6 => array( + 'var' => 'startTime', + 'type' => TType::I64, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['headerNames'])) { + $this->headerNames = $vals['headerNames']; + } + if (isset($vals['rows'])) { + $this->rows = $vals['rows']; + } + if (isset($vals['progressedPercentage'])) { + $this->progressedPercentage = $vals['progressedPercentage']; + } + if (isset($vals['status'])) { + $this->status = $vals['status']; + } + if (isset($vals['footerSummary'])) { + $this->footerSummary = $vals['footerSummary']; + } + if (isset($vals['startTime'])) { + $this->startTime = $vals['startTime']; + } + } + } + + public function getName() { + return 'TProgressUpdateResp'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::LST) { + $this->headerNames = array(); + $_size159 = 0; + $_etype162 = 0; + $xfer += $input->readListBegin($_etype162, $_size159); + for ($_i163 = 0; $_i163 < $_size159; ++$_i163) + { + $elem164 = null; + $xfer += $input->readString($elem164); + $this->headerNames []= $elem164; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::LST) { + $this->rows = array(); + $_size165 = 0; + $_etype168 = 0; + $xfer += $input->readListBegin($_etype168, $_size165); + for ($_i169 = 0; $_i169 < $_size165; ++$_i169) + { + $elem170 = null; + $elem170 = array(); + $_size171 = 0; + $_etype174 = 0; + $xfer += $input->readListBegin($_etype174, $_size171); + for ($_i175 = 0; $_i175 < $_size171; ++$_i175) + { + $elem176 = null; + $xfer += $input->readString($elem176); + $elem170 []= $elem176; + } + $xfer += $input->readListEnd(); + $this->rows []= $elem170; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::DOUBLE) { + $xfer += $input->readDouble($this->progressedPercentage); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->status); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->footerSummary); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->startTime); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('TProgressUpdateResp'); + if ($this->headerNames !== null) { + if (!is_array($this->headerNames)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('headerNames', TType::LST, 1); + { + $output->writeListBegin(TType::STRING, count($this->headerNames)); + { + foreach ($this->headerNames as $iter177) + { + $xfer += $output->writeString($iter177); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->rows !== null) { + if (!is_array($this->rows)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('rows', TType::LST, 2); + { + $output->writeListBegin(TType::LST, count($this->rows)); + { + foreach ($this->rows as $iter178) + { + { + $output->writeListBegin(TType::STRING, count($iter178)); + { + foreach ($iter178 as $iter179) + { + $xfer += $output->writeString($iter179); + } + } + $output->writeListEnd(); + } + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->progressedPercentage !== null) { + $xfer += $output->writeFieldBegin('progressedPercentage', TType::DOUBLE, 3); + $xfer += $output->writeDouble($this->progressedPercentage); + $xfer += $output->writeFieldEnd(); + } + if ($this->status !== null) { + $xfer += $output->writeFieldBegin('status', TType::I32, 4); + $xfer += $output->writeI32($this->status); + $xfer += $output->writeFieldEnd(); + } + if ($this->footerSummary !== null) { + $xfer += $output->writeFieldBegin('footerSummary', TType::STRING, 5); + $xfer += $output->writeString($this->footerSummary); + $xfer += $output->writeFieldEnd(); + } + if ($this->startTime !== null) { + $xfer += $output->writeFieldBegin('startTime', TType::I64, 6); + $xfer += $output->writeI64($this->startTime); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + final class Constant extends \Thrift\Type\TConstant { static protected $PRIMITIVE_TYPES; static protected $COMPLEX_TYPES; diff --git service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService-remote service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService-remote index 62b87a3..b2f6461 100755 --- service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService-remote +++ service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService-remote @@ -45,6 +45,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help': print(' TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req)') print(' TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req)') print(' TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req)') + print(' TProgressUpdateResp GetProgressUpdate(TProgressUpdateReq req)') print('') sys.exit(0) @@ -227,6 +228,12 @@ elif cmd == 'RenewDelegationToken': sys.exit(1) pp.pprint(client.RenewDelegationToken(eval(args[0]),)) +elif cmd == 'GetProgressUpdate': + if len(args) != 1: + print('GetProgressUpdate requires 1 args') + sys.exit(1) + pp.pprint(client.GetProgressUpdate(eval(args[0]),)) + else: print('Unrecognized method %s' % cmd) sys.exit(1) diff --git service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService.py service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService.py index 35f2ae4..d15f97f 100644 --- service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService.py +++ service-rpc/src/gen/thrift/gen-py/TCLIService/TCLIService.py @@ -166,6 +166,13 @@ def RenewDelegationToken(self, req): """ pass + def GetProgressUpdate(self, req): + """ + Parameters: + - req + """ + pass + class Client(Iface): def __init__(self, iprot, oprot=None): @@ -825,6 +832,37 @@ def recv_RenewDelegationToken(self): return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "RenewDelegationToken failed: unknown result") + def GetProgressUpdate(self, req): + """ + Parameters: + - req + """ + self.send_GetProgressUpdate(req) + return self.recv_GetProgressUpdate() + + def send_GetProgressUpdate(self, req): + self._oprot.writeMessageBegin('GetProgressUpdate', TMessageType.CALL, self._seqid) + args = GetProgressUpdate_args() + args.req = req + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_GetProgressUpdate(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = GetProgressUpdate_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "GetProgressUpdate failed: unknown result") + class Processor(Iface, TProcessor): def __init__(self, handler): @@ -851,6 +889,7 @@ def __init__(self, handler): self._processMap["GetDelegationToken"] = Processor.process_GetDelegationToken self._processMap["CancelDelegationToken"] = Processor.process_CancelDelegationToken self._processMap["RenewDelegationToken"] = Processor.process_RenewDelegationToken + self._processMap["GetProgressUpdate"] = Processor.process_GetProgressUpdate def process(self, iprot, oprot): (name, type, seqid) = iprot.readMessageBegin() @@ -1266,6 +1305,25 @@ def process_RenewDelegationToken(self, seqid, iprot, oprot): oprot.writeMessageEnd() oprot.trans.flush() + def process_GetProgressUpdate(self, seqid, iprot, oprot): + args = GetProgressUpdate_args() + args.read(iprot) + iprot.readMessageEnd() + result = GetProgressUpdate_result() + try: + result.success = self._handler.GetProgressUpdate(args.req) + msg_type = TMessageType.REPLY + except (TTransport.TTransportException, KeyboardInterrupt, SystemExit): + raise + except Exception as ex: + msg_type = TMessageType.EXCEPTION + logging.exception(ex) + result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error') + oprot.writeMessageBegin("GetProgressUpdate", msg_type, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + # HELPER FUNCTIONS AND STRUCTURES @@ -4019,3 +4077,134 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) + +class GetProgressUpdate_args: + """ + Attributes: + - req + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'req', (TProgressUpdateReq, TProgressUpdateReq.thrift_spec), None, ), # 1 + ) + + def __init__(self, req=None,): + self.req = req + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.req = TProgressUpdateReq() + self.req.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('GetProgressUpdate_args') + if self.req is not None: + oprot.writeFieldBegin('req', TType.STRUCT, 1) + self.req.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.req) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class GetProgressUpdate_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (TProgressUpdateResp, TProgressUpdateResp.thrift_spec), None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = TProgressUpdateResp() + self.success.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('GetProgressUpdate_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) diff --git service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py index fdf6b1f..59d0a34 100644 --- service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py +++ service-rpc/src/gen/thrift/gen-py/TCLIService/ttypes.py @@ -393,6 +393,38 @@ class TFetchOrientation: "FETCH_LAST": 5, } +class TJobExecutionStatus: + SUBMITTED = 0 + INITING = 1 + RUNNING = 2 + SUCCEEDED = 3 + KILLED = 4 + FAILED = 5 + ERROR = 6 + NOT_AVAILABLE = 7 + + _VALUES_TO_NAMES = { + 0: "SUBMITTED", + 1: "INITING", + 2: "RUNNING", + 3: "SUCCEEDED", + 4: "KILLED", + 5: "FAILED", + 6: "ERROR", + 7: "NOT_AVAILABLE", + } + + _NAMES_TO_VALUES = { + "SUBMITTED": 0, + "INITING": 1, + "RUNNING": 2, + "SUCCEEDED": 3, + "KILLED": 4, + "FAILED": 5, + "ERROR": 6, + "NOT_AVAILABLE": 7, + } + class TTypeQualifierValue: """ @@ -7369,3 +7401,237 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) + +class TProgressUpdateReq: + """ + Attributes: + - operationHandle + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'operationHandle', (TOperationHandle, TOperationHandle.thrift_spec), None, ), # 1 + ) + + def __init__(self, operationHandle=None,): + self.operationHandle = operationHandle + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.operationHandle = TOperationHandle() + self.operationHandle.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TProgressUpdateReq') + if self.operationHandle is not None: + oprot.writeFieldBegin('operationHandle', TType.STRUCT, 1) + self.operationHandle.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.operationHandle is None: + raise TProtocol.TProtocolException(message='Required field operationHandle is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.operationHandle) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TProgressUpdateResp: + """ + Attributes: + - headerNames + - rows + - progressedPercentage + - status + - footerSummary + - startTime + """ + + thrift_spec = ( + None, # 0 + (1, TType.LIST, 'headerNames', (TType.STRING,None), None, ), # 1 + (2, TType.LIST, 'rows', (TType.LIST,(TType.STRING,None)), None, ), # 2 + (3, TType.DOUBLE, 'progressedPercentage', None, None, ), # 3 + (4, TType.I32, 'status', None, None, ), # 4 + (5, TType.STRING, 'footerSummary', None, None, ), # 5 + (6, TType.I64, 'startTime', None, None, ), # 6 + ) + + def __init__(self, headerNames=None, rows=None, progressedPercentage=None, status=None, footerSummary=None, startTime=None,): + self.headerNames = headerNames + self.rows = rows + self.progressedPercentage = progressedPercentage + self.status = status + self.footerSummary = footerSummary + self.startTime = startTime + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.LIST: + self.headerNames = [] + (_etype162, _size159) = iprot.readListBegin() + for _i163 in xrange(_size159): + _elem164 = iprot.readString() + self.headerNames.append(_elem164) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.rows = [] + (_etype168, _size165) = iprot.readListBegin() + for _i169 in xrange(_size165): + _elem170 = [] + (_etype174, _size171) = iprot.readListBegin() + for _i175 in xrange(_size171): + _elem176 = iprot.readString() + _elem170.append(_elem176) + iprot.readListEnd() + self.rows.append(_elem170) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.DOUBLE: + self.progressedPercentage = iprot.readDouble() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.status = iprot.readI32() + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.footerSummary = iprot.readString() + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.I64: + self.startTime = iprot.readI64() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TProgressUpdateResp') + if self.headerNames is not None: + oprot.writeFieldBegin('headerNames', TType.LIST, 1) + oprot.writeListBegin(TType.STRING, len(self.headerNames)) + for iter177 in self.headerNames: + oprot.writeString(iter177) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.rows is not None: + oprot.writeFieldBegin('rows', TType.LIST, 2) + oprot.writeListBegin(TType.LIST, len(self.rows)) + for iter178 in self.rows: + oprot.writeListBegin(TType.STRING, len(iter178)) + for iter179 in iter178: + oprot.writeString(iter179) + oprot.writeListEnd() + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.progressedPercentage is not None: + oprot.writeFieldBegin('progressedPercentage', TType.DOUBLE, 3) + oprot.writeDouble(self.progressedPercentage) + oprot.writeFieldEnd() + if self.status is not None: + oprot.writeFieldBegin('status', TType.I32, 4) + oprot.writeI32(self.status) + oprot.writeFieldEnd() + if self.footerSummary is not None: + oprot.writeFieldBegin('footerSummary', TType.STRING, 5) + oprot.writeString(self.footerSummary) + oprot.writeFieldEnd() + if self.startTime is not None: + oprot.writeFieldBegin('startTime', TType.I64, 6) + oprot.writeI64(self.startTime) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.headerNames is None: + raise TProtocol.TProtocolException(message='Required field headerNames is unset!') + if self.rows is None: + raise TProtocol.TProtocolException(message='Required field rows is unset!') + if self.progressedPercentage is None: + raise TProtocol.TProtocolException(message='Required field progressedPercentage is unset!') + if self.status is None: + raise TProtocol.TProtocolException(message='Required field status is unset!') + if self.footerSummary is None: + raise TProtocol.TProtocolException(message='Required field footerSummary is unset!') + if self.startTime is None: + raise TProtocol.TProtocolException(message='Required field startTime is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.headerNames) + value = (value * 31) ^ hash(self.rows) + value = (value * 31) ^ hash(self.progressedPercentage) + value = (value * 31) ^ hash(self.status) + value = (value * 31) ^ hash(self.footerSummary) + value = (value * 31) ^ hash(self.startTime) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) diff --git service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service.rb service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service.rb index a50fe25..bd2b94f 100644 --- service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service.rb +++ service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service.rb @@ -326,6 +326,21 @@ module TCLIService raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'RenewDelegationToken failed: unknown result') end + def GetProgressUpdate(req) + send_GetProgressUpdate(req) + return recv_GetProgressUpdate() + end + + def send_GetProgressUpdate(req) + send_message('GetProgressUpdate', GetProgressUpdate_args, :req => req) + end + + def recv_GetProgressUpdate() + result = receive_message(GetProgressUpdate_result) + return result.success unless result.success.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'GetProgressUpdate failed: unknown result') + end + end class Processor @@ -478,6 +493,13 @@ module TCLIService write_result(result, oprot, 'RenewDelegationToken', seqid) end + def process_GetProgressUpdate(seqid, iprot, oprot) + args = read_args(iprot, GetProgressUpdate_args) + result = GetProgressUpdate_result.new() + result.success = @handler.GetProgressUpdate(args.req) + write_result(result, oprot, 'GetProgressUpdate', seqid) + end + end # HELPER FUNCTIONS AND STRUCTURES @@ -1154,5 +1176,37 @@ module TCLIService ::Thrift::Struct.generate_accessors self end + class GetProgressUpdate_args + include ::Thrift::Struct, ::Thrift::Struct_Union + REQ = 1 + + FIELDS = { + REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::TProgressUpdateReq} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class GetProgressUpdate_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::TProgressUpdateResp} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + end diff --git service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb index 4b1854c..ef02acf 100644 --- service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb +++ service-rpc/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb @@ -148,6 +148,19 @@ module TFetchOrientation VALID_VALUES = Set.new([FETCH_NEXT, FETCH_PRIOR, FETCH_RELATIVE, FETCH_ABSOLUTE, FETCH_FIRST, FETCH_LAST]).freeze end +module TJobExecutionStatus + SUBMITTED = 0 + INITING = 1 + RUNNING = 2 + SUCCEEDED = 3 + KILLED = 4 + FAILED = 5 + ERROR = 6 + NOT_AVAILABLE = 7 + VALUE_MAP = {0 => "SUBMITTED", 1 => "INITING", 2 => "RUNNING", 3 => "SUCCEEDED", 4 => "KILLED", 5 => "FAILED", 6 => "ERROR", 7 => "NOT_AVAILABLE"} + VALID_VALUES = Set.new([SUBMITTED, INITING, RUNNING, SUCCEEDED, KILLED, FAILED, ERROR, NOT_AVAILABLE]).freeze +end + class TTypeQualifierValue < ::Thrift::Union include ::Thrift::Struct_Union class << self @@ -1867,3 +1880,55 @@ class TRenewDelegationTokenResp ::Thrift::Struct.generate_accessors self end +class TProgressUpdateReq + include ::Thrift::Struct, ::Thrift::Struct_Union + OPERATIONHANDLE = 1 + + FIELDS = { + OPERATIONHANDLE => {:type => ::Thrift::Types::STRUCT, :name => 'operationHandle', :class => ::TOperationHandle} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field operationHandle is unset!') unless @operationHandle + end + + ::Thrift::Struct.generate_accessors self +end + +class TProgressUpdateResp + include ::Thrift::Struct, ::Thrift::Struct_Union + HEADERNAMES = 1 + ROWS = 2 + PROGRESSEDPERCENTAGE = 3 + STATUS = 4 + FOOTERSUMMARY = 5 + STARTTIME = 6 + + FIELDS = { + HEADERNAMES => {:type => ::Thrift::Types::LIST, :name => 'headerNames', :element => {:type => ::Thrift::Types::STRING}}, + ROWS => {:type => ::Thrift::Types::LIST, :name => 'rows', :element => {:type => ::Thrift::Types::LIST, :element => {:type => ::Thrift::Types::STRING}}}, + PROGRESSEDPERCENTAGE => {:type => ::Thrift::Types::DOUBLE, :name => 'progressedPercentage'}, + STATUS => {:type => ::Thrift::Types::I32, :name => 'status', :enum_class => ::TJobExecutionStatus}, + FOOTERSUMMARY => {:type => ::Thrift::Types::STRING, :name => 'footerSummary'}, + STARTTIME => {:type => ::Thrift::Types::I64, :name => 'startTime'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field headerNames is unset!') unless @headerNames + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field rows is unset!') unless @rows + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field progressedPercentage is unset!') unless @progressedPercentage + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field status is unset!') unless @status + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field footerSummary is unset!') unless @footerSummary + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field startTime is unset!') unless @startTime + unless @status.nil? || ::TJobExecutionStatus::VALID_VALUES.include?(@status) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field status!') + end + end + + ::Thrift::Struct.generate_accessors self +end + diff --git service/src/java/org/apache/hive/service/cli/CLIService.java service/src/java/org/apache/hive/service/cli/CLIService.java index de44ecb..1603113 100644 --- service/src/java/org/apache/hive/service/cli/CLIService.java +++ service/src/java/org/apache/hive/service/cli/CLIService.java @@ -46,6 +46,8 @@ import org.apache.hive.service.server.HiveServer2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hive.ql.log.ProgressMonitor; +import org.apache.hive.service.cli.session.HiveSession; /** * CLIService. @@ -255,8 +257,11 @@ public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType @Override public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, Map confOverlay) throws HiveSQLException { - OperationHandle opHandle = - sessionManager.getSession(sessionHandle).executeStatement(statement, confOverlay); + HiveSession session = sessionManager.getSession(sessionHandle); + // need to reset the monitor, as operation handle is not available down stream, Ideally the + // monitor should be associated with the operation handle. + session.getSessionState().updateProgress(null); + OperationHandle opHandle = session.executeStatement(statement, confOverlay); LOG.debug(sessionHandle + ": executeStatement()"); return opHandle; } @@ -267,9 +272,11 @@ public OperationHandle executeStatement(SessionHandle sessionHandle, String stat @Override public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, Map confOverlay, long queryTimeout) throws HiveSQLException { - OperationHandle opHandle = - sessionManager.getSession(sessionHandle).executeStatement(statement, confOverlay, - queryTimeout); + HiveSession session = sessionManager.getSession(sessionHandle); + // need to reset the monitor, as operation handle is not available down stream, Ideally the + // monitor should be associated with the operation handle. + session.getSessionState().updateProgress(null); + OperationHandle opHandle = session.executeStatement(statement, confOverlay, queryTimeout); LOG.debug(sessionHandle + ": executeStatement()"); return opHandle; } @@ -410,13 +417,35 @@ public OperationHandle getCrossReference(SessionHandle sessionHandle, String foreignSchema, String foreignTable) throws HiveSQLException { OperationHandle opHandle = sessionManager.getSession(sessionHandle) - .getCrossReference(primaryCatalog, primarySchema, primaryTable, + .getCrossReference(primaryCatalog, primarySchema, primaryTable, foreignCatalog, foreignSchema, foreignTable); LOG.debug(sessionHandle + ": getCrossReference()"); return opHandle; } - + + @Override + public JobProgressUpdate progressUpdate(OperationHandle operationHandle) throws HiveSQLException { + if ("tez".equals(hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE))) { + Operation operation = sessionManager.getOperationManager().getOperation(operationHandle); + if (OperationType.EXECUTE_STATEMENT.equals(operation.getType())) { + SessionState sessionState = operation.getParentSession().getSessionState(); + try { + while (sessionState.progressMonitor() == null && !operation.isFinished()) { + Thread.sleep(100L); // sleep for 100 ms + } + } catch (InterruptedException e) { + LOG.warn("Error while getting progress update", e); + } + + if (sessionState.progressMonitor() != null) { + return new JobProgressUpdate(sessionState.progressMonitor()); + } + } + } + return new JobProgressUpdate(ProgressMonitor.NULL); + } + /* (non-Javadoc) * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) */ diff --git service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java index 2a3bcca..db5d8eb 100644 --- service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java +++ service/src/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java @@ -226,4 +226,9 @@ public OperationHandle getCrossReference(SessionHandle sessionHandle, return cliService.getCrossReference(sessionHandle, primaryCatalog, primarySchema, primaryTable, foreignCatalog, foreignSchema, foreignTable); } + + @Override + public JobProgressUpdate progressUpdate(OperationHandle operationHandle) throws HiveSQLException { + return cliService.progressUpdate(operationHandle); + } } diff --git service/src/java/org/apache/hive/service/cli/ICLIService.java service/src/java/org/apache/hive/service/cli/ICLIService.java index fef772d..d311268 100644 --- service/src/java/org/apache/hive/service/cli/ICLIService.java +++ service/src/java/org/apache/hive/service/cli/ICLIService.java @@ -109,4 +109,6 @@ OperationHandle getCrossReference(SessionHandle sessionHandle, String primaryCatalog, String primarySchema, String primaryTable, String foreignCatalog, String foreignSchema, String foreignTable) throws HiveSQLException; + + JobProgressUpdate progressUpdate(OperationHandle operationHandle) throws HiveSQLException; } diff --git service/src/java/org/apache/hive/service/cli/JobExecutionStatus.java service/src/java/org/apache/hive/service/cli/JobExecutionStatus.java new file mode 100644 index 0000000..65f09a1 --- /dev/null +++ service/src/java/org/apache/hive/service/cli/JobExecutionStatus.java @@ -0,0 +1,37 @@ +package org.apache.hive.service.cli; + +public enum JobExecutionStatus { + SUBMITTED((short) 0), + + INITING((short) 1), + + RUNNING((short) 2), + + SUCCEEDED((short) 3), + + KILLED((short) 4), + + FAILED((short) 5), + + ERROR((short) 6), + + NOT_AVAILABLE((short) 7); + + private final short executionStatusOrdinal; + + JobExecutionStatus(short executionStatusOrdinal) { + this.executionStatusOrdinal = executionStatusOrdinal; + } + + public short toExecutionStatus() { + return executionStatusOrdinal; + } + + public static JobExecutionStatus fromString(String input) { + for (JobExecutionStatus status : values()) { + if (status.name().equals(input)) + return status; + } + return NOT_AVAILABLE; + } +} diff --git service/src/java/org/apache/hive/service/cli/JobProgressUpdate.java service/src/java/org/apache/hive/service/cli/JobProgressUpdate.java new file mode 100644 index 0000000..6d624e5 --- /dev/null +++ service/src/java/org/apache/hive/service/cli/JobProgressUpdate.java @@ -0,0 +1,39 @@ +package org.apache.hive.service.cli; + +import org.apache.hadoop.hive.ql.log.ProgressMonitor; + +import java.util.List; + +public class JobProgressUpdate { + public final double progressedPercentage; + public final JobExecutionStatus jobExecutionStatus; + public final String footerSummary; + public final long startTimeMillis; + private final List headers; + private final List> rows; + + + JobProgressUpdate(ProgressMonitor monitor) { + this(monitor.headers(), monitor.rows(), monitor.footerSummary(), monitor.progressedPercentage(), + JobExecutionStatus.fromString(monitor.executionStatus()), monitor.startTime()); + } + + public JobProgressUpdate(List headers, List> rows, String footerSummary, + double progressedPercentage, JobExecutionStatus jobExecutionStatus, + long startTimeMillis) { + this.progressedPercentage = progressedPercentage; + this.jobExecutionStatus = jobExecutionStatus; + this.footerSummary = footerSummary; + this.startTimeMillis = startTimeMillis; + this.headers = headers; + this.rows = rows; + } + + public List headers() { + return headers; + } + + public List> rows() { + return rows; + } +} diff --git service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java index 85b82b6..b4decf9 100644 --- service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java +++ service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java @@ -32,7 +32,6 @@ import javax.security.sasl.SaslException; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.auth.PlainSaslHelper; import org.apache.hive.service.cli.CLIServiceClient; @@ -42,6 +41,7 @@ import org.apache.hive.service.cli.GetInfoValue; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.ICLIService; +import org.apache.hive.service.cli.JobProgressUpdate; import org.apache.hive.service.cli.OperationHandle; import org.apache.hive.service.cli.OperationStatus; import org.apache.hive.service.cli.RowSet; @@ -235,6 +235,11 @@ public OperationHandle getCrossReference(SessionHandle sessionHandle, return cliService.getCrossReference(sessionHandle, primaryCatalog, primarySchema, primaryTable, foreignCatalog, foreignSchema, foreignTable); } + + @Override + public JobProgressUpdate progressUpdate(OperationHandle operationHandle) throws HiveSQLException { + return cliService.progressUpdate(operationHandle); + } } protected RetryingThriftCLIServiceClient(HiveConf conf) { diff --git service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 2938338..bfdc2a6 100644 --- service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -24,7 +24,6 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import javax.security.auth.login.LoginException; @@ -33,19 +32,18 @@ import org.apache.hadoop.hive.common.ServerUtils; import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.security.SaslRpcServer.AuthMethod; import org.apache.hive.service.AbstractService; import org.apache.hive.service.ServiceException; import org.apache.hive.service.ServiceUtils; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.auth.TSetIpAddressProcessor; -import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.FetchOrientation; import org.apache.hive.service.cli.FetchType; import org.apache.hive.service.cli.GetInfoType; import org.apache.hive.service.cli.GetInfoValue; import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.JobProgressUpdate; import org.apache.hive.service.cli.OperationHandle; import org.apache.hive.service.cli.OperationStatus; import org.apache.hive.service.cli.RowSet; @@ -91,8 +89,11 @@ import org.apache.hive.service.rpc.thrift.TGetTablesResp; import org.apache.hive.service.rpc.thrift.TGetTypeInfoReq; import org.apache.hive.service.rpc.thrift.TGetTypeInfoResp; +import org.apache.hive.service.rpc.thrift.TJobExecutionStatus; import org.apache.hive.service.rpc.thrift.TOpenSessionReq; import org.apache.hive.service.rpc.thrift.TOpenSessionResp; +import org.apache.hive.service.rpc.thrift.TProgressUpdateReq; +import org.apache.hive.service.rpc.thrift.TProgressUpdateResp; import org.apache.hive.service.rpc.thrift.TProtocolVersion; import org.apache.hive.service.rpc.thrift.TRenewDelegationTokenReq; import org.apache.hive.service.rpc.thrift.TRenewDelegationTokenResp; @@ -746,7 +747,7 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) } return resp; } - + @Override public abstract void run(); @@ -795,4 +796,22 @@ private String getProxyUser(String realUser, Map sessionConf, LOG.debug("Verified proxy user: " + proxyUser); return proxyUser; } + + @Override + public TProgressUpdateResp GetProgressUpdate(TProgressUpdateReq req) throws TException { + OperationHandle opHandle = new OperationHandle(req.getOperationHandle()); + try { + TProgressUpdateResp response = new TProgressUpdateResp(); + JobProgressUpdate jobProgressUpdate = cliService.progressUpdate(opHandle); + response.setHeaderNames(jobProgressUpdate.headers()); + response.setRows(jobProgressUpdate.rows()); + response.setStartTime(jobProgressUpdate.startTimeMillis); + response.setProgressedPercentage(jobProgressUpdate.progressedPercentage); + response.setStatus(TJobExecutionStatus.findByValue(jobProgressUpdate.jobExecutionStatus.toExecutionStatus())); + response.setFooterSummary(jobProgressUpdate.footerSummary); + return response; + } catch (HiveSQLException e) { + throw new TException(e); + } + } } diff --git service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java index 9805641..9bbc9ee 100644 --- service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java +++ service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java @@ -29,6 +29,8 @@ import org.apache.hive.service.cli.GetInfoType; import org.apache.hive.service.cli.GetInfoValue; import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.JobExecutionStatus; +import org.apache.hive.service.cli.JobProgressUpdate; import org.apache.hive.service.cli.OperationHandle; import org.apache.hive.service.cli.OperationState; import org.apache.hive.service.cli.OperationStatus; @@ -77,6 +79,8 @@ import org.apache.hive.service.rpc.thrift.TGetTypeInfoResp; import org.apache.hive.service.rpc.thrift.TOpenSessionReq; import org.apache.hive.service.rpc.thrift.TOpenSessionResp; +import org.apache.hive.service.rpc.thrift.TProgressUpdateReq; +import org.apache.hive.service.rpc.thrift.TProgressUpdateResp; import org.apache.hive.service.rpc.thrift.TProtocolVersion; import org.apache.hive.service.rpc.thrift.TRenewDelegationTokenReq; import org.apache.hive.service.rpc.thrift.TRenewDelegationTokenResp; @@ -552,4 +556,21 @@ public OperationHandle getCrossReference(SessionHandle sessionHandle, throw new HiveSQLException(e); } } + + @Override + public JobProgressUpdate progressUpdate(OperationHandle operationHandle) throws HiveSQLException { + try { + TProgressUpdateResp tProgressUpdateResp = cliService.GetProgressUpdate(new TProgressUpdateReq(operationHandle.toTOperationHandle())); + return new JobProgressUpdate( + tProgressUpdateResp.getHeaderNames(), + tProgressUpdateResp.getRows(), + tProgressUpdateResp.getFooterSummary(), + tProgressUpdateResp.getProgressedPercentage(), + JobExecutionStatus.fromString(tProgressUpdateResp.getStatus().name()), + tProgressUpdateResp.getStartTime() + ); + } catch (TException e) { + throw new HiveSQLException(e); + } + } }