private static List<Column> getColumns(StatementClient client, Consumer<QueryStats> progressCallback) throws SQLException { while (client.isRunning()) { QueryStatusInfo results = client.currentStatusInfo(); progressCallback.accept(QueryStats.create(results.getId(), results.getStats())); List<Column> columns = results.getColumns(); if (columns != null) { return columns; } client.advance(); } verify(client.isFinished()); QueryStatusInfo results = client.finalStatusInfo(); if (results.getError() == null) { throw new SQLException(format("Query has no columns (#%s)", results.getId())); } throw resultsException(results); }
private void renderUpdate(PrintStream out, QueryStatusInfo results) { String status = results.getUpdateType(); if (results.getUpdateCount() != null) { long count = results.getUpdateCount(); status += format(": %s row%s", count, (count != 1) ? "s" : ""); } out.println(status); discardResults(); }
static SQLException resultsException(QueryStatusInfo results) { QueryError error = requireNonNull(results.getError()); String message = format("Query failed (#%s): %s", results.getId(), error.getMessage()); Throwable cause = (error.getFailureInfo() == null) ? null : error.getFailureInfo().toException(); return new SQLException(message, error.getSqlState(), error.getErrorCode(), cause); }
if (client.isRunning() || (client.isFinished() && client.finalStatusInfo().getError() == null)) { QueryStatusInfo results = client.isRunning() ? client.currentStatusInfo() : client.finalStatusInfo(); if (results.getUpdateType() != null) { renderUpdate(errorChannel, results); else if (results.getColumns() == null) { errorChannel.printf("Query %s has no columns\n", results.getId()); return false; renderResults(out, outputFormat, interactive, results.getColumns()); new PrintStreamWarningsPrinter(System.err).print(client.finalStatusInfo().getWarnings(), true, true); statusPrinter.printFinalInfo(); warningsPrinter.print(client.finalStatusInfo().getWarnings(), true, true); if (client.finalStatusInfo().getError() != null) { renderFailure(errorChannel); return false;
progressCallback.accept(QueryStats.create(results.getId(), results.getStats())); warningsManager.addWarnings(results.getWarnings()); Iterable<List<Object>> data = client.currentData().getData(); progressCallback.accept(QueryStats.create(results.getId(), results.getStats())); warningsManager.addWarnings(results.getWarnings()); if (results.getError() != null) { throw new RuntimeException(resultsException(results));
QueryError error = client.finalStatusInfo().getError(); if (results.getUpdateType() != null) { resultsSession.setUpdateType(results.getUpdateType()); if (results.getUpdateCount() != null) { resultsSession.setUpdateCount(results.getUpdateCount()); return new ResultWithQueryId<>(new QueryId(results.getId()), result);
private void printQueryInfo(QueryStatusInfo results, WarningsPrinter warningsPrinter) StatementStats stats = results.getStats(); Duration wallTime = nanosSince(start); results.getId(), stats.getState(), nodes, reprintLine(querySummary); String url = results.getInfoUri().toString(); if (debug && (url.length() < terminalWidth)) { reprintLine(url); results.getId(), stats.getState(), reprintLine(querySummary); warningsPrinter.print(results.getWarnings(), false, false);
if (client.isFinished()) { QueryStatusInfo finalStatusInfo = client.finalStatusInfo(); if (finalStatusInfo.getError() != null) { throw resultsException(finalStatusInfo); if (client.currentStatusInfo().getUpdateType() == null) { currentResult.set(resultSet); return true; Long updateCount = client.finalStatusInfo().getUpdateCount(); currentUpdateCount.set((updateCount != null) ? updateCount : 0); currentUpdateType.set(client.finalStatusInfo().getUpdateType()); warningsManager.addWarnings(client.finalStatusInfo().getWarnings()); return false;
StatementStats stats = results.getStats(); results.getId(), stats.getState(), nodes, out.println(results.getInfoUri().toString());
private StatementStats execute(ClientSession session, String query, Consumer<QueryData> queryDataConsumer, Consumer<QueryError> queryErrorConsumer) { // start query try (StatementClient client = newStatementClient(okHttpClient, session, query)) { // read query output while (client.isRunning()) { queryDataConsumer.accept(client.currentData()); if (!client.advance()) { break; } } // verify final state if (client.isClientAborted()) { throw new IllegalStateException("Query aborted by user"); } if (client.isClientError()) { throw new IllegalStateException("Query is gone (server restarted?)"); } verify(client.isFinished()); QueryError resultsError = client.finalStatusInfo().getError(); if (resultsError != null) { queryErrorConsumer.accept(resultsError); } return client.finalStatusInfo().getStats(); } }
@Override public void addResults(QueryStatusInfo statusInfo, QueryData data) { if (types.get() == null && statusInfo.getColumns() != null) { types.set(getTypes(statusInfo.getColumns())); } if (data.getData() != null) { checkState(types.get() != null, "data received without types"); rows.addAll(transform(data.getData(), dataToRow(types.get()))); } }
PrestoResultSet(StatementClient client, long maxRows, Consumer<QueryStats> progressCallback, WarningsManager warningsManager) throws SQLException { this.client = requireNonNull(client, "client is null"); requireNonNull(progressCallback, "progressCallback is null"); this.sessionTimeZone = DateTimeZone.forID(client.getTimeZone().getId()); this.queryId = client.currentStatusInfo().getId(); List<Column> columns = getColumns(client, progressCallback); this.fieldMap = getFieldMap(columns); this.columnInfoList = getColumnInfo(columns); this.resultSetMetaData = new PrestoResultSetMetaData(columnInfoList); this.warningsManager = requireNonNull(warningsManager, "warningsManager is null"); this.results = flatten(new ResultsPageIterator(client, progressCallback, warningsManager), maxRows); }
private static boolean isQuery(StatementClient client) { String updateType; if (client.isRunning()) { updateType = client.currentStatusInfo().getUpdateType(); } else { updateType = client.finalStatusInfo().getUpdateType(); } return updateType == null; }
private void processInitialStatusUpdates(WarningsPrinter warningsPrinter) { while (client.isRunning() && (client.currentData().getData() == null)) { warningsPrinter.print(client.currentStatusInfo().getWarnings(), true, false); client.advance(); } List<Warning> warnings; if (client.isRunning()) { warnings = client.currentStatusInfo().getWarnings(); } else { warnings = client.finalStatusInfo().getWarnings(); } warningsPrinter.print(warnings, false, true); }
if (client.isRunning() || (client.isFinished() && client.finalStatusInfo().getError() == null)) { QueryStatusInfo results = client.isRunning() ? client.currentStatusInfo() : client.finalStatusInfo(); if (results.getUpdateType() != null) { renderUpdate(errorChannel, results); else if (results.getColumns() == null) { errorChannel.printf("Query %s has no columns\n", results.getId()); return false; renderResults(out, outputFormat, interactive, results.getColumns()); new PrintStreamWarningsPrinter(System.err).print(client.finalStatusInfo().getWarnings(), true, true); statusPrinter.printFinalInfo(); warningsPrinter.print(client.finalStatusInfo().getWarnings(), true, true); if (client.finalStatusInfo().getError() != null) { renderFailure(errorChannel); return false;
private void printQueryInfo(QueryStatusInfo results, WarningsPrinter warningsPrinter) StatementStats stats = results.getStats(); Duration wallTime = nanosSince(start); results.getId(), stats.getState(), nodes, reprintLine(querySummary); String url = results.getInfoUri().toString(); if (debug && (url.length() < terminalWidth)) { reprintLine(url); results.getId(), stats.getState(), reprintLine(querySummary); warningsPrinter.print(results.getWarnings(), false, false);
public void renderFailure(PrintStream out) { QueryStatusInfo results = client.finalStatusInfo(); QueryError error = results.getError(); checkState(error != null); out.printf("Query %s failed: %s%n", results.getId(), error.getMessage()); if (debug && (error.getFailureInfo() != null)) { error.getFailureInfo().toException().printStackTrace(out); } if (error.getErrorLocation() != null) { renderErrorLocation(client.getQuery(), error.getErrorLocation(), out); } out.println(); }
StatementStats stats = results.getStats(); results.getId(), stats.getState(), nodes, out.println(results.getInfoUri().toString());
@Override public void addResults(QueryStatusInfo statusInfo, QueryData data) { if (types.get() == null && statusInfo.getColumns() != null) { types.set(getTypes(statusInfo.getColumns())); } if (data.getData() != null) { checkState(types.get() != null, "Data without types received!"); List<Column> columns = statusInfo.getColumns(); for (List<Object> fields : data.getData()) { ImmutableMap.Builder<String, Object> builder = ImmutableMap.builder(); for (int i = 0; i < fields.size(); i++) { Type type = types.get().get(i); Object value = convertValue(fields.get(i), type); if (value != null) { builder.put(columns.get(i).getName(), value); } } producer.send(new KeyedMessage<>(topicName, count.getAndIncrement(), builder.build())); } } }