@Override protected void finalize() throws Throwable { // CHECKSTYLE:ON // TODO Log an error if transaction is not closed (See FLUO-486) close(false); }
@Override public Bytes get(Bytes row, Column column) { checkIfOpen(); // TODO cache? precache? return get(row, Collections.singleton(column)).get(column); }
public TransactionImpl(Environment env, Notification trigger) { this(env, trigger, allocateTimestamp(env).getTxTimestamp()); }
@Override public Map<Column, Bytes> get(Bytes row, Set<Column> columns) { checkIfOpen(); return getImpl(row, columns); }
@Override public void delete(Bytes row, Column col) throws AlreadySetException { checkIfOpen(); Preconditions.checkNotNull(row); Preconditions.checkNotNull(col); set(row, col, DELETE); }
deleteWeakRow(); stats.setFinishTime(System.currentTimeMillis()); return; CommitData cd = createCommitData(); if (!preCommit(cd)) { readUnread(cd); throw new CommitException("Pre-commit failed"); if (commitPrimaryColumn(cd, commitStamp)) { long t3 = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); stats.setCommitTs(commitStamp.getTxTimestamp()); finishCommit(cd, commitStamp); long t4 = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); stats.setCommitTimes(t2 - t1, t3 - t2, t4 - t3);
AlreadyAcknowledgedException { checkIfOpen(); status = TxStatus.COMMIT_STARTED; prewrite(cd.prow, cd.pcol, cd.pval, cd.prow, cd.pcol, isTriggerRow(cd.prow)); if (checkForAckCollision(pcm)) { throw new AlreadyAcknowledgedException(); for (Entry<Bytes, Map<Column, Bytes>> rowUpdates : updates.entrySet()) { ConditionalFlutation cm = null; boolean isTriggerRow = isTriggerRow(rowUpdates.getKey()); prewrite(rowUpdates.getKey(), colUpdates.getKey(), colUpdates.getValue(), cd.prow, cd.pcol, isTriggerRow); } else { prewrite(cm, colUpdates.getKey(), colUpdates.getValue(), cd.prow, cd.pcol, isTriggerRow); } else { ackCollision |= checkForAckCollision(result.getMutation()); cd.addToRejected(row, updates.get(row).keySet()); rollback(cd);
@Override public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { try { ByteArrayInputStream bais = new ByteArrayInputStream(context.getConfiguration().get(PROPS_CONF_KEY) .getBytes("UTF-8")); PropertiesConfiguration props = new PropertiesConfiguration(); props.load(bais); env = new Environment(new FluoConfiguration(props)); ti = new TransactionImpl(env, context.getConfiguration().getLong(TIMESTAMP_CONF_KEY, -1)); // TODO this uses non public Accumulo API! RangeInputSplit ris = (RangeInputSplit) split; Span span = SpanUtil.toSpan(ris.getRange()); ScannerConfiguration sc = new ScannerConfiguration().setSpan(span); for (String fam : context.getConfiguration().getStrings(FAMS_CONF_KEY, new String[0])) { sc.fetchColumnFamily(Bytes.of(fam)); } rowIter = ti.get(sc); } catch (Exception e) { throw new IOException(e); } }
writeWeakNotifications(commitTs); boolean isTrigger = isTriggerRow(cd.prow) && cd.pcol.equals(notification.getColumn()); getTransactorID())); ConditionalMutation delLockMutation = new ConditionalFlutation(env, cd.prow, lockCheck);
String status = "UNKNOWN"; try { txi = new TransactionImpl(env); tx = txi; if (TracingTransaction.isTracingEnabled()) { if (txi != null) { try { txi.getStats().report(status, loader.getClass()); } finally { tx.close();
@Override public RowIterator get(ScannerConfiguration config) { // TODO log something better (see fluo-425) log.trace("txid: {} get(ScannerConfiguration)", txid); return tx.get(config); }
@Override public void close() { log.trace("txid: {} close()", txid); if (summaryLog.isTraceEnabled()) { TxStats stats = tx.getStats(); String className = "N/A"; if (clazz != null) { className = clazz.getSimpleName(); } // TODO log total # read, see fluo-426 summaryLog.trace( "txid: {} thread : {} time: {} ({} {} {}) #ret: {} #set: {} #collisions: {} " + "waitTime: {} committed: {} class: {}", txid, Thread.currentThread().getId(), stats.getTime(), stats.getPrecommitTime(), stats.getCommitPrimaryTime(), stats.getFinishCommitTime(), stats.getEntriesReturned(), stats.getEntriesSet(), stats.getCollisions(), stats.getLockWaitTime(), committed, className); } tx.close(); }
@Override public Snapshot newSnapshot() { TransactionImpl tx = new TransactionImpl(env); if (TracingTransaction.isTracingEnabled()) { return new TracingTransaction(tx); } return tx; }
@Override public Map<Bytes, Map<Column, Bytes>> get(Collection<Bytes> rows, Set<Column> columns) { checkIfOpen(); if (rows.size() == 0 || columns.size() == 0) { return Collections.emptyMap(); } env.getSharedResources().getVisCache().validate(columns); ParallelSnapshotScanner pss = new ParallelSnapshotScanner(rows, columns, env, startTs, stats); Map<Bytes, Map<Column, Bytes>> ret = pss.scan(); for (Entry<Bytes, Map<Column, Bytes>> entry : ret.entrySet()) { updateColumnsRead(entry.getKey(), entry.getValue().keySet()); } return ret; }
@Override public void commit() throws CommitException { try { tx.commit(); committed = true; log.trace("txid: {} commit() -> SUCCESSFUL commitTs: {}", txid, tx.getStats().getCommitTs()); } catch (CommitException ce) { log.trace("txid: {} commit() -> UNSUCCESSFUL commitTs: {}", txid, tx.getStats().getCommitTs()); if (!log.isTraceEnabled() && notification != null) { collisionLog.trace("txid: {} trigger: {} {} {}", txid, notification.getRow(), notification.getColumn(), notification.getTimestamp()); } if (!log.isTraceEnabled() && clazz != null) { collisionLog.trace("txid: {} class: {}", txid, clazz.getName()); } collisionLog.trace("txid: {} collisions: {}", txid, tx.getStats().getRejected()); throw ce; } }
@Override public void setWeakNotification(Bytes row, Column col) { checkIfOpen(); Preconditions.checkNotNull(row); Preconditions.checkNotNull(col); if (!env.getWeakObservers().containsKey(col)) { throw new IllegalArgumentException("Column not configured for weak notifications " + col); } env.getSharedResources().getVisCache().validate(col); Set<Column> columns = weakNotifications.get(row); if (columns == null) { columns = new HashSet<>(); weakNotifications.put(row, columns); } columns.add(col); }
@Override public void delete(Bytes row, Column col) throws AlreadySetException { if (log.isTraceEnabled()) { log.trace("txid: {} delete({}, {})", txid, enc(row), enc(col)); } tx.delete(row, col); }
@Override public void commit() { super.commit(); // wait for any async mutations that transaction write to flush env.getSharedResources().getBatchWriter().waitForAsyncFlush(); } };
@Override public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { try { ByteArrayInputStream bais = new ByteArrayInputStream(context.getConfiguration().get(PROPS_CONF_KEY) .getBytes("UTF-8")); PropertiesConfiguration props = new PropertiesConfiguration(); props.load(bais); env = new Environment(new FluoConfiguration(props)); ti = new TransactionImpl(env, context.getConfiguration().getLong(TIMESTAMP_CONF_KEY, -1)); // TODO this uses non public Accumulo API! RangeInputSplit ris = (RangeInputSplit) split; Span span = SpanUtil.toSpan(ris.getRange()); ScannerConfiguration sc = new ScannerConfiguration().setSpan(span); for (String fam : context.getConfiguration().getStrings(FAMS_CONF_KEY, new String[0])) { sc.fetchColumnFamily(Bytes.of(fam)); } rowIter = ti.get(sc); } catch (Exception e) { throw new IOException(e); } }
UtilWaitThread.sleep(sleepTime); txi = new TransactionImpl(env, notification); tx = txi; if (TracingTransaction.isTracingEnabled()) { if (txi != null) { try { txi.getStats().report(status.toString(), observer.getClass()); } finally { tx.close();