protected static KuduTable createFourTabletsTableWithNineRows(String tableName) throws Exception { CreateTableBuilder builder = new CreateTableBuilder(); PartialRow splitRow = basicSchema.newPartialRow(); for (int i : KEYS) { splitRow.addInt(0, i); builder.addSplitRow(splitRow); } KuduTable table = createTable(tableName, basicSchema, builder); AsyncKuduSession session = client.newSession(); // create a table with on empty tablet and 3 tablets of 3 rows each for (int key1 : KEYS) { for (int key2 = 1; key2 <= 3; key2++) { Insert insert = table.newInsert(); PartialRow row = insert.getRow(); row.addInt(0, key1 + key2); row.addInt(1, key1); row.addInt(2, key2); row.addString(3, "a string"); row.addBoolean(4, true); session.apply(insert).join(DEFAULT_SLEEP); } } session.close().join(DEFAULT_SLEEP); return table; }
/** * Blocking call with a different behavior based on the flush mode. PleaseThrottleException is * managed by this method and will not be thrown, unlike {@link AsyncKuduSession#apply}. * * <p> * * <ul> * <li>AUTO_FLUSH_SYNC: the call returns when the operation is persisted, else it throws an * exception. * <li>AUTO_FLUSH_BACKGROUND: the call returns when the operation has been added to the buffer. * This call should normally perform only fast in-memory operations but it may have to wait * when the buffer is full and there's another buffer being flushed. Row errors can be * checked by calling {@link #countPendingErrors()} and can be retrieved by calling {@link * #getPendingErrors()}. * <li>MANUAL_FLUSH: the call returns when the operation has been added to the buffer, else it * throws an exception such as a NonRecoverableException if the buffer is full. * </ul> * * @param operation operation to apply * @return an OperationResponse for the applied Operation * @throws Exception if anything went wrong */ public OperationResponse apply(Operation operation) throws Exception { while (true) { try { Deferred<OperationResponse> d = session.apply(operation); if (getFlushMode() == FlushMode.AUTO_FLUSH_SYNC) { return d.join(getTimeoutMillis()); } break; } catch (PleaseThrottleException ex) { try { ex.getDeferred().join(getTimeoutMillis()); } catch (Exception e) { // This is the error response from the buffer that was flushing, // we can't do much with it at this point. LOG.error("Previous batch had this exception", e); } } catch (Exception e) { throw e; } } return null; }
@Override public RowErrorsAndOverflowStatus getPendingErrors() { return session.getPendingErrors(); }
@Override public int countPendingErrors() { return session.countPendingErrors(); }
@Override public void setIgnoreAllDuplicateRows(boolean ignoreAllDuplicateRows) { session.setIgnoreAllDuplicateRows(ignoreAllDuplicateRows); }
@Override public boolean isIgnoreAllDuplicateRows() { return session.isIgnoreAllDuplicateRows(); }
@Override public void setExternalConsistencyMode(ExternalConsistencyMode consistencyMode) { session.setExternalConsistencyMode(consistencyMode); }
@Override public boolean hasPendingOperations() { return session.hasPendingOperations(); }
@Override public FlushMode getFlushMode() { return session.getFlushMode(); }
@Override public void setTimeoutMillis(long timeout) { session.setTimeoutMillis(timeout); }
@Override public long getTimeoutMillis() { return session.getTimeoutMillis(); }
@Override public void setFlushInterval(int interval) { session.setFlushInterval(interval); }
@Override public void setMutationBufferLowWatermark(float mutationBufferLowWatermarkPercentage) { session.setMutationBufferLowWatermark(mutationBufferLowWatermarkPercentage); }
@Override public void setMutationBufferSpace(int size) { session.setMutationBufferSpace(size); }
@Override public void setFlushMode(AsyncKuduSession.FlushMode flushMode) { session.setFlushMode(flushMode); }
@Override public boolean isClosed() { return session.isClosed(); }
/** * Blocking call that flushes the buffers (see {@link #flush()} and closes the sessions. * * @return List of OperationResponse, one per operation that was flushed * @throws Exception if anything went wrong. If it's an issue with some or all batches, it will be * of type DeferredGroupException. */ public List<OperationResponse> close() throws Exception { return session.close().join(getTimeoutMillis()); }