/** * Add timer metrics to {@link DistributedFileSystem#mkdirs(Path, FsPermission)} */ public boolean mkdirs(Path f, FsPermission permission) throws IOException { try (Closeable context = new TimerContextWithLog(mkdirTimer.time(), "mkdirs", f, permission)) { return super.mkdirs (f, permission); } }
@Override public ContextAwareTimer newMetric(MetricContext context, String name) { return new ContextAwareTimer(context, name); }
@Override protected void onClose() { StringBuilder message = new StringBuilder(); message.append("========================").append("\n"); message.append("Statistics for FileSystem: ").append(getUri()).append("\n"); message.append("------------------------").append("\n"); message.append("method\tcalls\tmean time(ns)\t99 percentile(ns)").append("\n"); for (ContextAwareTimer timer : this.allTimers) { if (timer.getCount() > 0) { message.append(timer.getName()).append("\t").append(timer.getCount()).append("\t"). append(timer.getSnapshot().getMean()).append("\t").append(timer.getSnapshot().get99thPercentile()).append("\n"); } } message.append("------------------------").append("\n"); log.info(message.toString()); super.onClose(); }
@Override public void update(long duration, TimeUnit unit) { super.update(duration, unit); if (this.parentTimer.isPresent()) { this.parentTimer.get().update(duration, unit); } }
/** * End the previous stage and record the time spent in that stage. */ public void endStage() { if (this.currentStage != null) { long time = System.currentTimeMillis() - this.currentStageStart; this.timings.add(new Stage(this.currentStage, time)); if (reportAsMetrics && submitter.getMetricContext().isPresent()) { String timerName = submitter.getNamespace() + "." + name + "." + this.currentStage; submitter.getMetricContext().get().timer(timerName).update(time, TimeUnit.MILLISECONDS); } } this.currentStage = null; }
/** * Add timer metrics to {@link DistributedFileSystem#open(Path, int)} */ public FSDataInputStream open(Path f, int bufferSize) throws IOException { try (Closeable context = new TimerContextWithLog(this.openTimer.time(), "open", f, bufferSize)) { return super.open(f, bufferSize); } }
protected MetricContext(String name, MetricContext parent, List<Tag<?>> tags, boolean isRoot) throws NameConflictException { Preconditions.checkArgument(!Strings.isNullOrEmpty(name)); this.closer = Closer.create(); try { this.innerMetricContext = this.closer.register(new InnerMetricContext(this, name, parent, tags)); } catch(ExecutionException ee) { throw Throwables.propagate(ee); } this.contextAwareMetricsSet = Sets.newConcurrentHashSet(); this.notificationTargets = Maps.newConcurrentMap(); this.executorServiceOptional = Optional.absent(); this.notificationTimer = new ContextAwareTimer(this, GOBBLIN_METRICS_NOTIFICATIONS_TIMER_NAME); register(this.notificationTimer); if (!isRoot) { RootMetricContext.get().addMetricContext(this); } }
/** * Add timer metrics to {@link DistributedFileSystem#setTimes(Path, long, long)} */ public void setTimes (Path f, long t, long a) throws IOException { try (Closeable context = new TimerContextWithLog(this.setTimesTimer.time(), "setTimes", f, t, a)) { super.setTimes(f, t, a); } }
/** * Add timer metrics to {@link DistributedFileSystem#concat(Path, Path[])} */ public void concat (Path trg, Path [] psrcs) throws IOException { try (Closeable context = new TimerContextWithLog(this.concatTimer.time(), "concat", trg, psrcs)) { super.concat(trg, psrcs); } } }
/** * Add timer metrics to {@link FileSystem#listFiles(Path, boolean)} */ public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException { try (Closeable context = new TimerContextWithLog(this.listFilesTimer.time(), "listFiles", f, recursive)) { return super.listFiles(f, recursive); } }
/** * Add timer metrics to {@link DistributedFileSystem#append(Path, int, Progressable)} */ public FSDataOutputStream append (Path p, final int bufferSize, Progressable progress) throws IOException { try (Closeable context = new TimerContextWithLog(this.appendTimer.time(), "append", p)) { return super.append(p, bufferSize, progress); } }
/** * Send a notification to all targets of this context and to the parent of this context. * @param notification {@link gobblin.metrics.notification.Notification} to send. */ public void sendNotification(final Notification notification) { ContextAwareTimer.Context timer = this.notificationTimer.time(); if(!this.notificationTargets.isEmpty()) { for (final Map.Entry<UUID, Function<Notification, Void>> entry : this.notificationTargets.entrySet()) { try { entry.getValue().apply(notification); } catch (RuntimeException exception) { LOG.warn("RuntimeException when running notification target. Skipping.", exception); } } } if(getParent().isPresent()) { getParent().get().sendNotification(notification); } timer.stop(); }
private void addSchemaProperties(Path path, HiveRegistrationUnit hiveUnit) throws IOException { Preconditions.checkArgument(this.fs.getFileStatus(path).isDirectory(), path + " is not a directory."); Path schemaFile = new Path(path, this.schemaFileName); if (this.useSchemaFile) { hiveUnit.setSerDeProp(SCHEMA_URL, schemaFile.toString()); } else { Schema schema ; try (Timer.Context context = metricContext.timer(HIVE_SPEC_SCHEMA_READING_TIMER).time()) { schema = getDirectorySchema(path); } try (Timer.Context context = metricContext.timer(HIVE_SPEC_SCHEMA_WRITING_TIMER).time()) { addSchemaFromAvroFile(schema, schemaFile, hiveUnit); } } }
/** * Add timer metrics to {@link DistributedFileSystem#rename(Path, Path)} */ public boolean rename (Path src, Path dst) throws IOException { try (Closeable context = new TimerContextWithLog(renameTimer.time(), "rename", src, dst)) { return super.rename(src, dst); } }
/** * Add timer metrics to {@link DistributedFileSystem#delete(Path, boolean)} */ public boolean delete (Path f, boolean recursive) throws IOException { try (Closeable context = new TimerContextWithLog(deleteTimer.time(), "delete", f, recursive)) { return super.delete (f, recursive); } }
/** * Add timer metrics to {@link DistributedFileSystem#setOwner(Path, String, String)} */ public void setOwner(Path f, String user, String group) throws IOException { try (Closeable context = new TimerContextWithLog(this.setOwnerTimer.time(), "setOwner", f, user, group)) { super.setOwner(f, user, group); } }
/** * Add timer metrics to {@link DistributedFileSystem#getFileStatus(Path)} */ public FileStatus getFileStatus (Path f) throws IOException { try (Closeable context = new TimerContextWithLog(this.getFileStatusTimer.time(), "getFileStatus", f)) { return super.getFileStatus(f); } }
/** * Add timer metrics to {@link DistributedFileSystem#setPermission(Path, FsPermission)} */ public void setPermission (Path f, final FsPermission permission) throws IOException { try (Closeable context = new TimerContextWithLog(this.setPermissionTimer.time(), "setPermission", f, permission)) { super.setPermission(f, permission); } }
/** * Add timer metrics to {@link DistributedFileSystem#create(Path, FsPermission, boolean, int, short, long, Progressable)} */ public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { try (Closeable context = new TimerContextWithLog(this.createTimer.time(), "create", f, permission, overwrite, bufferSize, replication, blockSize, progress)) { return super.create(f, permission, overwrite, bufferSize, replication, blockSize, progress); } }
@Override public boolean existsTable(String dbName, String tableName) throws IOException { if (this.optimizedChecks && this.tableAndDbExistenceCache.getIfPresent(dbName + ":" + tableName ) != null ) { return true; } try (AutoReturnableObject<IMetaStoreClient> client = this.clientPool.getClient()) { try (Timer.Context context = this.metricContext.timer(TABLE_EXISTS).time()) { return client.get().tableExists(dbName, tableName); } } catch (TException e) { throw new IOException(String.format("Unable to check existence of table %s in db %s", tableName, dbName), e); } }