You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2021/01/26 19:45:51 UTC

[hbase-thirdparty] branch master updated: HBASE-24802 make a drop-in compatible impl of htrace APIs that does not do anything

This is an automated email from the ASF dual-hosted git repository.

busbey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase-thirdparty.git


The following commit(s) were added to refs/heads/master by this push:
     new a8e3629  HBASE-24802 make a drop-in compatible impl of htrace APIs that does not do anything
a8e3629 is described below

commit a8e3629618642fd50daa31cc027bc674ab659711
Author: Sean Busbey <bu...@apache.org>
AuthorDate: Sun Aug 16 00:05:30 2020 -0500

    HBASE-24802 make a drop-in compatible impl of htrace APIs that does not do anything
    
    closes #36
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 dev-support/jenkins/Jenkinsfile                    | 237 +++++++++++++++++++--
 dev-support/jenkins/swap_htrace_jar.sh             |  34 +++
 hbase-noop-htrace/pom.xml                          | 103 +++++++++
 .../java/org/apache/htrace/core/AlwaysSampler.java |  32 +++
 .../java/org/apache/htrace/core/CountSampler.java  |  33 +++
 .../apache/htrace/core/HTraceConfiguration.java    |  67 ++++++
 .../htrace/core/JavaPropertyConfiguration.java     |  59 +++++
 .../apache/htrace/core/LocalFileSpanReceiver.java  |  47 ++++
 .../java/org/apache/htrace/core/MilliSpan.java     | 185 ++++++++++++++++
 .../java/org/apache/htrace/core/NeverSampler.java  |  32 +++
 .../org/apache/htrace/core/POJOSpanReceiver.java   |  46 ++++
 .../org/apache/htrace/core/ProbabilitySampler.java |  35 +++
 .../main/java/org/apache/htrace/core/Sampler.java  |  64 ++++++
 .../htrace/core/ScheduledTraceExecutorService.java |  66 ++++++
 .../src/main/java/org/apache/htrace/core/Span.java | 164 ++++++++++++++
 .../main/java/org/apache/htrace/core/SpanId.java   |  89 ++++++++
 .../java/org/apache/htrace/core/SpanReceiver.java  | 102 +++++++++
 .../htrace/core/StandardOutSpanReceiver.java       |  36 ++++
 .../org/apache/htrace/core/TimelineAnnotation.java |  36 ++++
 .../java/org/apache/htrace/core/TraceCallable.java |  40 ++++
 .../apache/htrace/core/TraceExecutorService.java   | 111 ++++++++++
 .../java/org/apache/htrace/core/TraceRunnable.java |  51 +++++
 .../java/org/apache/htrace/core/TraceScope.java    |  82 +++++++
 .../main/java/org/apache/htrace/core/Tracer.java   | 231 ++++++++++++++++++++
 .../main/java/org/apache/htrace/core/TracerId.java |  55 +++++
 .../java/org/apache/htrace/core/TracerPool.java    | 155 ++++++++++++++
 .../TestHBaseTestingUtilityWithHTraceNoop.java     |  62 ++++++
 pom.xml                                            |   1 +
 28 files changed, 2239 insertions(+), 16 deletions(-)

diff --git a/dev-support/jenkins/Jenkinsfile b/dev-support/jenkins/Jenkinsfile
index 86d2bfe..cdcd7cc 100644
--- a/dev-support/jenkins/Jenkinsfile
+++ b/dev-support/jenkins/Jenkinsfile
@@ -30,6 +30,20 @@ pipeline {
         skipDefaultCheckout()
     }
 
+    parameters {
+        booleanParam(name: 'DEBUG',
+               defaultValue: false,
+               description: 'Print extra outputs for debugging the jenkins job and yetus')
+        // the hbase and hadoop versions listed here need to match the matrix axes in the test
+        // section. it's not currently possible to reuse a single array for both purposes.
+        choice(name: 'HBASE_VERSION',
+               choices: ['all', '2.2.6', '2.3.3'],
+               description: 'HBase releases to test. default is everything in the list.')
+        choice(name: 'HADOOP_VERSION',
+               choices: ['all', '3.2.1', '2.10.0'],
+               description: 'Hadoop versions to run each hbase version on. default is everything in the list.')
+    }
+
     environment {
         SRC_REL = 'src'
         PATCH_REL = 'output'
@@ -39,26 +53,26 @@ pipeline {
         DOCKERFILE_REL = "${SRC_REL}/dev-support/jenkins/Dockerfile"
         YETUS_DRIVER_REL = "${SRC_REL}/dev-support/jenkins/jenkins_precommit_github_yetus.sh"
         ARCHIVE_PATTERN_LIST = '*.dump'
-        BUILD_URL_ARTIFACTS = "artifact/${WORKDIR_REL}/${PATCH_REL}"
         SET_JAVA_HOME = '/usr/local/openjdk-8'
-        WORKDIR_REL = 'yetus-precommit-check'
-        WORKDIR = "${WORKSPACE}/${WORKDIR_REL}"
-        SOURCEDIR = "${WORKDIR}/${SRC_REL}"
-        PATCHDIR = "${WORKDIR}/${PATCH_REL}"
-        DOCKERFILE = "${WORKDIR}/${DOCKERFILE_REL}"
-        YETUS_DRIVER = "${WORKDIR}/${YETUS_DRIVER_REL}"
-        YETUSDIR = "${WORKDIR}/${YETUS_REL}"
         PLUGINS = 'all'
     }
 
-    parameters {
-        booleanParam(name: 'DEBUG',
-               defaultValue: false,
-               description: 'Print extra outputs for debugging the jenkins job and yetus')
-    }
 
     stages {
         stage ('precommit checks') {
+            when {
+                changeRequest()
+            }
+            environment {
+                WORKDIR_REL = 'yetus-precommit-check'
+                WORKDIR = "${WORKSPACE}/${WORKDIR_REL}"
+                SOURCEDIR = "${WORKDIR}/${SRC_REL}"
+                PATCHDIR = "${WORKDIR}/${PATCH_REL}"
+                DOCKERFILE = "${WORKDIR}/${DOCKERFILE_REL}"
+                YETUS_DRIVER = "${WORKDIR}/${YETUS_DRIVER_REL}"
+                YETUSDIR = "${WORKDIR}/${YETUS_REL}"
+                BUILD_URL_ARTIFACTS = "artifact/${WORKDIR_REL}/${PATCH_REL}"
+            }
             steps {
                 dir("${SOURCEDIR}") {
                     checkout scm
@@ -124,18 +138,209 @@ pipeline {
                 }
             }
         }
+        stage ('noop htrace drop in') {
+            when {
+                anyOf {
+                    changeset "hbase-noop-htrace/**"
+                    changeset "dev-support/jenkins/Jenkinsfile"
+                }
+            }
+            tools {
+                // this needs to be set to the jdk that ought to be used to build releases on the branch the Jenkinsfile is stored in.
+                jdk "jdk_1.8_latest"
+            }
+            stages {
+                stage ('setup') {
+                    tools {
+                        maven 'maven_latest'
+                    }
+                    environment {
+                        WORKDIR = "${WORKSPACE}/htrace-noop"
+                        CACHE_DIR = "${WORKSPACE}/cache"
+                    }
+                    steps {
+                        dir ("htrace-noop") {
+                            dir ("component") {
+                                echo 'Build the htrace replacement artifact.'
+                                checkout scm
+                                sh 'mvn -DskipTests -pl hbase-noop-htrace clean package'
+                            }
+                            dir ("tools") {
+                                echo 'Downloading enabling scripts from main hbase repo.'
+                                sh '''#!/usr/bin/env bash
+                                    set -oe
+                                    declare script
+                                    declare -a needed_files
+                                    needed_files=( \
+                                        hbase_nightly_pseudo-distributed-test.sh \
+                                        jenkins-scripts/cache-apache-project-artifact.sh \
+                                    )
+                                    for script in "${needed_files[@]}"; do
+                                        curl -L -O https://raw.githubusercontent.com/apache/hbase/HEAD/dev-support/"${script}"
+                                        chmod +x "$(basename "${script}")"
+                                    done
+                                '''
+                            }
+                            stash name: 'scripts', includes: "tools/hbase_nightly_pseudo-distributed-test.sh"
+                            dir ("hbase") {
+                                script {
+                                    def hbase_versions = [ params.HBASE_VERSION ]
+                                    if (params.HBASE_VERSION == 'all') {
+                                        // this set needs to match the matrix axes below
+                                        hbase_versions = [ '2.2.6', '2.3.3' ]
+                                    }
+                                    hbase_versions.each {
+                                        def hbase_version = it
+                                        sh """#!/usr/bin/env bash
+                                            set -e
+                                            set -x
+                                            mkdir -p "downloads/hbase-${hbase_version}"
+                                            mkdir -p "${CACHE_DIR}"
+                                            echo 'downloading hbase version ${hbase_version}'
+                                            '${WORKDIR}/tools/cache-apache-project-artifact.sh' \
+                                                    --working-dir '${WORKDIR}/hbase/downloads/hbase-${hbase_version}' \
+                                                    --keys 'https://downloads.apache.org/hbase/KEYS' \
+                                                    '${CACHE_DIR}/hbase-${hbase_version}-bin.tar.gz' \
+                                                    'hbase/${hbase_version}/hbase-${hbase_version}-bin.tar.gz'
+                                            mkdir 'hbase-${hbase_version}'
+                                            declare noop_htrace
+                                            noop_htrace="\$(ls -1 '${WORKDIR}/component/hbase-noop-htrace/target/'hbase-noop-htrace-*.jar | head -n 1)"
+                                            if [ -z "\${noop_htrace}" ]; then
+                                              echo "failed to find htrace noop replacement. did building step work?" >&2
+                                              exit 1
+                                            fi
+                                            '${WORKDIR}/component/dev-support/jenkins/swap_htrace_jar.sh' '${WORKDIR}/hbase/hbase-${hbase_version}' '${CACHE_DIR}/hbase-${hbase_version}-bin.tar.gz' "\${noop_htrace}"
+                                        """
+                                        stash name: "hbase-${hbase_version}", includes: "hbase-${hbase_version}/**"
+                                    }
+                                }
+                            }
+                            dir ("hadoop") {
+                                script {
+                                    def hadoop_versions = [ params.HADOOP_VERSION ]
+                                    if (params.HADOOP_VERSION == 'all') {
+                                        // this set needs to match the matrix axes below
+                                        hadoop_versions = [ '3.2.1', '2.10.0']
+                                    }
+                                    hadoop_versions.each {
+                                        def hadoop_version = it
+                                        sh """#!/usr/bin/env bash
+                                            set -e
+                                            set -x
+                                            mkdir -p "downloads/hadoop-${hadoop_version}"
+                                            mkdir -p "${CACHE_DIR}"
+                                            echo "downloading hadoop version ${hadoop_version}"
+                                            "${WORKDIR}/tools/cache-apache-project-artifact.sh" \
+                                                    --working-dir "${WORKDIR}/hadoop/downloads/hadoop-${hadoop_version}" \
+                                                    --keys 'https://downloads.apache.org/hadoop/common/KEYS' \
+                                                    "${CACHE_DIR}/hadoop-${hadoop_version}-bin.tar.gz" \
+                                                    "hadoop/common/hadoop-${hadoop_version}/hadoop-${hadoop_version}.tar.gz"
+                                            mkdir "hadoop-${hadoop_version}"
+                                            declare noop_htrace
+                                            noop_htrace="\$(ls -1 "${WORKDIR}"/component/hbase-noop-htrace/target/hbase-noop-htrace-*.jar | head -n 1)"
+                                            if [ -z "\${noop_htrace}" ]; then
+                                              echo "failed to find htrace noop replacement. did building step work?" >&2
+                                              exit 1
+                                            fi
+                                            '${WORKDIR}/component/dev-support/jenkins/swap_htrace_jar.sh' "${WORKDIR}/hadoop/hadoop-${hadoop_version}" "${CACHE_DIR}/hadoop-${hadoop_version}-bin.tar.gz" "\${noop_htrace}"
+                                        """
+                                        stash name: "hadoop-${hadoop_version}", includes: "hadoop-${hadoop_version}/**"
+                                    }
+                                }
+                            }
+                        }
+                    }
+                    post {
+                        cleanup {
+                            // clean up the working area but don't delete the download cache
+                            dir ("htrace-noop") {
+                                deleteDir()
+                            }
+                        }
+                    }
+                }
+                stage ("test htrace drop in replacement") {
+                    matrix {
+                        agent {
+                            label 'hbase'
+                        }
+                        axes {
+                            axis {
+                                name 'HBASE'
+                                values '2.2.6', '2.3.3'
+                            }
+                            axis {
+                                name 'HADOOP'
+                                values '3.2.1', '2.10.0'
+                            }
+                        }
+                        when {
+                            allOf {
+                                anyOf {
+                                    expression { params.HBASE_VERSION == 'all' }
+                                    expression { params.HBASE_VERSION == env.HBASE }
+                                }
+                                anyOf {
+                                    expression { params.HADOOP_VERSION == 'all' }
+                                    expression { params.HADOOP_VERSION == env.HADOOP }
+                                }
+                            }
+                        }
+                        stages {
+                            stage ("test a specific hbase on a specific hadoop") {
+                                steps {
+                                    unstash 'scripts'
+                                    unstash "hbase-${env.HBASE}"
+                                    unstash "hadoop-${env.HADOOP}"
+                                    sh '''#!/usr/bin/env bash
+                                        set -eo
+                                        set -x
+                                        mkdir -p "hbase-${HBASE}.hadoop-${HADOOP}"
+                                        ./tools/hbase_nightly_pseudo-distributed-test.sh \
+                                                --single-process \
+                                                --working-dir "hbase-${HBASE}.hadoop-${HADOOP}" \
+                                                "hbase-${HBASE}" \
+                                                "hadoop-${HADOOP}/bin/hadoop" \
+                                                "hadoop-${HADOOP}"/share/hadoop/yarn/timelineservice \
+                                                "hadoop-${HADOOP}"/share/hadoop/yarn/test/hadoop-yarn-server-tests-*-tests.jar \
+                                                "hadoop-${HADOOP}"/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-*-tests.jar \
+                                                "hadoop-${HADOOP}"/bin/mapred \
+                                    '''
+                                }
+                                post {
+                                    failure {
+                                        sh '''#!/usr/bin/env bash
+                                            set -eo
+                                            set -x
+                                            find "hbase-${HBASE}.hadoop-${HADOOP}" \
+                                                    "hbase-${HBASE}" "hadoop-${HADOOP}" \
+                                                    -type d -name logs | \
+                                                    xargs zip -r "hbase-${HBASE}.hadoop-${HADOOP}.logs.zip"
+                                        '''
+                                        archiveArtifacts artifacts: "hbase-${env.HBASE}.hadoop-${env.HADOOP}.logs.zip"
+                                    }
+                                    cleanup {
+                                        deleteDir()
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        }
     }
 
     post {
         // Jenkins pipeline jobs fill slaves on PRs without this :(
-        cleanup() {
+        cleanup {
             script {
                 sh label: 'Cleanup workspace', script: '''#!/bin/bash -e
                     # See HADOOP-13951
                     chmod -R u+rxw "${WORKSPACE}"
-                    '''
-                deleteDir()
+                '''
             }
+            // we purposefully don't do a top level workspace cleanup so that we can reuse downloads
         }
     }
 }
diff --git a/dev-support/jenkins/swap_htrace_jar.sh b/dev-support/jenkins/swap_htrace_jar.sh
new file mode 100755
index 0000000..4aa8fc8
--- /dev/null
+++ b/dev-support/jenkins/swap_htrace_jar.sh
@@ -0,0 +1,34 @@
+#!/usr/bin/env bash
+# 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.
+
+# Call it like ./swap_htrace_jar.sh /some/place/to/target /a/path/to/component.tar.gz /a/path/to/hbase-noop-htrace.jar
+set -e
+if [ -n "${DEBUG}" ]; then
+  set -x
+fi
+declare destination="$1"
+declare tarball="$2"
+declare noop_htrace="$3"
+echo "unpack the tarball, but skip htrace artifacts."
+tar -xzf "${tarball}" --strip-components=1 --exclude 'htrace*.jar' -C "${destination}"
+echo "insert a copy of our replacement artifact for htrace."
+for htrace_location in $(tar -tzf "${tarball}" | grep -E 'htrace.*jar' ); do
+  htrace_location="$(dirname "${htrace_location#*/}")"
+  echo "        placing htrace jar at ${htrace_location}"
+  cp "${noop_htrace}" "${destination}/${htrace_location}"
+done;
diff --git a/hbase-noop-htrace/pom.xml b/hbase-noop-htrace/pom.xml
new file mode 100644
index 0000000..46f75de
--- /dev/null
+++ b/hbase-noop-htrace/pom.xml
@@ -0,0 +1,103 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <!--
+/**
+ * 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.
+ */
+
+-->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hbase.thirdparty</groupId>
+    <artifactId>hbase-thirdparty</artifactId>
+    <version>3.4.2-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>hbase-noop-htrace</artifactId>
+  <name>Apache HBase Drop-in noop HTrace replacement</name>
+  <description>
+    Implmeents the HTrace APIs with noops and nonsense aimed at a) removing CVE flagged transitive
+    dependencies and b) favoring the JIT optimizing it away.
+  </description>
+  <properties>
+    <maven.test.redirectTestOutputToFile>true</maven.test.redirectTestOutputToFile>
+    <!-- This library is meant to work with whatever HBase and Hadoop combos we still list
+         as recommended. The version here is what we'll test out of the box if you run mvn test
+      -->
+    <hbase.stable.version>2.3.3</hbase.stable.version>
+    <!-- should match the slf4j-api version used in the above hbase -->
+    <slf4j.version>1.7.25</slf4j.version>
+    <!-- This is also used by branch-1, so make sure we're jdk7 compatible -->
+    <compileSource>1.7</compileSource>
+    <!-- These maybe need to match the main repo -->
+    <maven.checkstyle.version>3.1.0</maven.checkstyle.version>
+    <checkstyle.version>8.28</checkstyle.version>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>4.12</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-shaded-testing-util</artifactId>
+      <version>${hbase.stable.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.htrace</groupId>
+          <artifactId>htrace-core4</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- TODO why does hbase-shaded testing-util need this but not declare it? -->
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven.checkstyle.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-checkstyle</artifactId>
+            <version>${hbase.stable.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>com.puppycrawl.tools</groupId>
+            <artifactId>checkstyle</artifactId>
+            <version>${checkstyle.version}</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <configLocation>hbase/checkstyle.xml</configLocation>
+          <suppressionsLocation>hbase/checkstyle-suppressions.xml</suppressionsLocation>
+          <includeTestSourceDirectory>true</includeTestSourceDirectory>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/AlwaysSampler.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/AlwaysSampler.java
new file mode 100644
index 0000000..8d5a296
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/AlwaysSampler.java
@@ -0,0 +1,32 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * A Sampler that always returns true.
+ */
+public final class AlwaysSampler extends Sampler {
+  public static final AlwaysSampler INSTANCE = new AlwaysSampler(null);
+
+  public AlwaysSampler(HTraceConfiguration conf) {
+  }
+
+  @Override
+  public boolean next() {
+    return true;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/CountSampler.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/CountSampler.java
new file mode 100644
index 0000000..3a55f2d
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/CountSampler.java
@@ -0,0 +1,33 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * Sampler that returns true every N calls. Specify the frequency interval by configuring a
+ * {@code long} value for {@link #SAMPLER_FREQUENCY_CONF_KEY}.
+ */
+public class CountSampler extends Sampler {
+  public final static String SAMPLER_FREQUENCY_CONF_KEY = "sampler.frequency";
+
+  public CountSampler(HTraceConfiguration conf) {
+  }
+
+  @Override
+  public boolean next() {
+    return false;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/HTraceConfiguration.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/HTraceConfiguration.java
new file mode 100644
index 0000000..f207399
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/HTraceConfiguration.java
@@ -0,0 +1,67 @@
+/*
+ * 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.htrace.core;
+
+import java.util.Map;
+
+/**
+ * Wrapper which integrating applications should implement in order
+ * to provide tracing configuration.
+ */
+public abstract class HTraceConfiguration {
+
+  /**
+   * An empty HTrace configuration.
+   */
+  public static final HTraceConfiguration EMPTY = new HTraceConfiguration() {
+    @Override
+    public String get(String key) {
+      return null;
+    }
+    @Override
+    public String get(String key, String defaultValue) {
+      return defaultValue;
+    }
+  };
+
+  /**
+   * Create an HTrace configuration from a map.
+   *
+   * @param conf    The map to create the configuration from.
+   * @return        The new configuration.
+   */
+  public static HTraceConfiguration fromMap(Map<String, String> conf) {
+    return EMPTY;
+  }
+
+  public static HTraceConfiguration fromKeyValuePairs(String... pairs) {
+    return EMPTY;
+  }
+
+  public abstract String get(String key);
+
+  public abstract String get(String key, String defaultValue);
+
+  public boolean getBoolean(String key, boolean defaultValue) {
+    return defaultValue;
+  }
+
+  public int getInt(String key, int defaultVal) {
+    return defaultVal;
+  }
+
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/JavaPropertyConfiguration.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/JavaPropertyConfiguration.java
new file mode 100644
index 0000000..f7a9777
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/JavaPropertyConfiguration.java
@@ -0,0 +1,59 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * This is an implementation of HTraceConfiguration which draws its properties
+ * from global Java Properties.
+ */
+public final class JavaPropertyConfiguration extends HTraceConfiguration {
+
+  public static class Builder {
+
+    public Builder() {
+    }
+
+    public Builder clearPrefixes() {
+      return this;
+    }
+
+    public Builder addPrefix(String prefix) {
+      return this;
+    }
+
+    JavaPropertyConfiguration build() {
+      return Holder.SINGLETON;
+    }
+  }
+
+  private JavaPropertyConfiguration() {
+  }
+
+  @Override
+  public String get(String key) {
+    return null;
+  }
+
+  @Override
+  public String get(String key, String defaultValue) {
+    return defaultValue;
+  }
+
+  private static final class Holder {
+    public static final JavaPropertyConfiguration SINGLETON = new JavaPropertyConfiguration();
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java
new file mode 100644
index 0000000..b176f48
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java
@@ -0,0 +1,47 @@
+/*
+ * 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.htrace.core;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+
+/**
+ * Writes the spans it receives to a local file.
+ */
+public class LocalFileSpanReceiver extends SpanReceiver {
+  public static final String PATH_KEY = "local.file.span.receiver.path";
+  public static final String CAPACITY_KEY = "local.file.span.receiver.capacity";
+  public static final int CAPACITY_DEFAULT = 5000;
+
+  public LocalFileSpanReceiver(HTraceConfiguration conf) {
+  }
+
+  @Override
+  public void receiveSpan(Span span) {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  public static String getUniqueLocalTraceFileName() {
+    String tmp = System.getProperty("java.io.tmpdir", "/tmp");
+    String nonce = UUID.randomUUID().toString();
+    return new File(tmp, nonce).getAbsolutePath();
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/MilliSpan.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/MilliSpan.java
new file mode 100644
index 0000000..04a1d00
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/MilliSpan.java
@@ -0,0 +1,185 @@
+/*
+ * 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.htrace.core;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Span implementation that stores its information in milliseconds since the
+ * epoch.
+ */
+public class MilliSpan implements Span {
+
+  @Override
+  public Span child(String childDescription) {
+    return Holder.SINGLETON;
+  }
+
+  /**
+   * The public interface for constructing a MilliSpan.
+   */
+  public static class Builder {
+
+    public Builder() {
+    }
+
+    public Builder begin(long begin) {
+      return this;
+    }
+
+    public Builder end(long end) {
+      return this;
+    }
+
+    public Builder description(String description) {
+      return this;
+    }
+
+    public Builder parents(SpanId[] parents) {
+      return this;
+    }
+
+    public Builder parents(List<SpanId> parentList) {
+      return this;
+    }
+
+    public Builder spanId(SpanId spanId) {
+      return this;
+    }
+
+    public Builder traceInfo(Map<String, String> traceInfo) {
+      return this;
+    }
+
+    public Builder tracerId(String tracerId) {
+      return this;
+    }
+
+    public Builder timeline(List<TimelineAnnotation> timeline) {
+      return this;
+    }
+
+    public MilliSpan build() {
+      return Holder.SINGLETON;
+    }
+  }
+
+  public MilliSpan() {
+  }
+
+  @Override
+  public synchronized void stop() {
+  }
+
+  protected long currentTimeMillis() {
+    return System.currentTimeMillis();
+  }
+
+  @Override
+  public synchronized boolean isRunning() {
+    return false;
+  }
+
+  @Override
+  public synchronized long getAccumulatedMillis() {
+    return 0;
+  }
+
+  @Override
+  public String toString() {
+    return "";
+  }
+
+  @Override
+  public String getDescription() {
+    return "";
+  }
+
+  @Override
+  public SpanId getSpanId() {
+    return SpanId.INVALID;
+  }
+
+  @Override
+  public SpanId[] getParents() {
+    return new SpanId[0];
+  }
+
+  @Override
+  public void setParents(SpanId[] parents) {
+  }
+
+  @Override
+  public long getStartTimeMillis() {
+    return 0;
+  }
+
+  @Override
+  public long getStopTimeMillis() {
+    return 0;
+  }
+
+  @Override
+  public void addKVAnnotation(String key, String value) {
+  }
+
+  @Override
+  public void addTimelineAnnotation(String msg) {
+  }
+
+  @Override
+  public Map<String, String> getKVAnnotations() {
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public List<TimelineAnnotation> getTimelineAnnotations() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public String getTracerId() {
+    return "";
+  }
+
+  @Override
+  public void setTracerId(String tracerId) {
+  }
+
+  @Override
+  public String toJson() {
+    return "";
+  }
+
+  /**
+   * N.B. had to break compatibility here because a jackson class was in the class heirarchy and
+   * method signature
+   */
+  public static class MilliSpanDeserializer {
+  }
+
+  public static MilliSpan fromJson(String json) throws IOException {
+    return Holder.SINGLETON;
+  }
+
+  private static final class Holder {
+    public static final MilliSpan SINGLETON = new MilliSpan();
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/NeverSampler.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/NeverSampler.java
new file mode 100644
index 0000000..60cc7d2
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/NeverSampler.java
@@ -0,0 +1,32 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * A Sampler that never returns true.
+ */
+public final class NeverSampler extends Sampler {
+  public static final NeverSampler INSTANCE = new NeverSampler(null);
+
+  public NeverSampler(HTraceConfiguration conf) {
+  }
+
+  @Override
+  public boolean next() {
+    return false;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java
new file mode 100644
index 0000000..aa31d71
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java
@@ -0,0 +1,46 @@
+/*
+ * 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.htrace.core;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * SpanReceiver for testing only that just collects the Span objects it
+ * receives. The spans it receives can be accessed with getSpans();
+ */
+public class POJOSpanReceiver extends SpanReceiver {
+
+  public POJOSpanReceiver(HTraceConfiguration conf) {
+  }
+
+  /**
+   * @return The spans this POJOSpanReceiver has received.
+   */
+  public Collection<Span> getSpans() {
+    return Collections.<Span>emptyList();
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void receiveSpan(Span span) {
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/ProbabilitySampler.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/ProbabilitySampler.java
new file mode 100644
index 0000000..cb208bb
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/ProbabilitySampler.java
@@ -0,0 +1,35 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * Sampler that returns true a certain percentage of the time. Specify the frequency interval by
+ * configuring a {@code double} value for {@link #SAMPLER_FRACTION_CONF_KEY}.
+ */
+public class ProbabilitySampler extends Sampler {
+  @SuppressWarnings("VisibilityModifier")
+  public final double threshold = 0.0d;
+  public final static String SAMPLER_FRACTION_CONF_KEY = "sampler.fraction";
+
+  public ProbabilitySampler(HTraceConfiguration conf) {
+  }
+
+  @Override
+  public boolean next() {
+    return false;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Sampler.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Sampler.java
new file mode 100644
index 0000000..1f613e3
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Sampler.java
@@ -0,0 +1,64 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * Extremely simple callback to determine the frequency that an action should
+ * be performed.
+ *
+ * <p>For example, the next() function may look like this:</p>
+ * <pre>
+ * <code>
+ * public boolean next() {
+ *   return Math.random() &gt; 0.5;
+ * }
+ * </code>
+ * </pre>
+ * <p>This would trace 50% of all gets, 75% of all puts and would not trace any
+ * other requests.</p>
+ */
+public abstract class Sampler {
+  /**
+   * A {@link Sampler} builder. It takes a {@link Sampler} class name and
+   * constructs an instance of that class, with the provided configuration.
+   */
+  public static class Builder {
+    public Builder(HTraceConfiguration conf) {
+    }
+
+    public Builder reset() {
+      return this;
+    }
+
+    public Builder className(String className) {
+      return this;
+    }
+
+    public Builder classLoader(ClassLoader classLoader) {
+      return this;
+    }
+
+    public Sampler build() {
+      return NEVER;
+    }
+  }
+
+  public static final Sampler ALWAYS = AlwaysSampler.INSTANCE;
+  public static final Sampler NEVER = NeverSampler.INSTANCE;
+
+  public abstract boolean next();
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/ScheduledTraceExecutorService.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/ScheduledTraceExecutorService.java
new file mode 100644
index 0000000..2c8d5c9
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/ScheduledTraceExecutorService.java
@@ -0,0 +1,66 @@
+/*
+ * 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.htrace.core;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A convenience wrapper around a {@link ScheduledExecutorService} for
+ * automatically propagating trace scopes to executable tasks.
+ * <p>
+ * Recurring tasks will use independent scopes per execution, but will all be
+ * tied to the same parent scope (if any).
+ */
+public class ScheduledTraceExecutorService extends TraceExecutorService
+    implements ScheduledExecutorService {
+  final ScheduledExecutorService impl;
+
+  ScheduledTraceExecutorService(Tracer tracer, String scopeName,
+      ScheduledExecutorService impl) {
+    super(tracer, scopeName, impl);
+    this.impl = impl;
+  }
+
+  @Override
+  public ScheduledFuture<?> schedule(Runnable command, long delay,
+      TimeUnit unit) {
+    return impl.schedule(command, delay, unit);
+  }
+
+  @Override
+  public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay,
+      TimeUnit unit) {
+    return impl.schedule(callable, delay, unit);
+  }
+
+  @Override
+  public ScheduledFuture<?> scheduleAtFixedRate(Runnable command,
+      long initialDelay, long period, TimeUnit unit) {
+    return impl.scheduleAtFixedRate(command, initialDelay, period, unit);
+  }
+
+  @Override
+  public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command,
+      long initialDelay, long delay, TimeUnit unit) {
+    return impl.scheduleWithFixedDelay(command, initialDelay, delay,
+        unit);
+  }
+
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Span.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Span.java
new file mode 100644
index 0000000..d97a41c
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Span.java
@@ -0,0 +1,164 @@
+/*
+ * 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.htrace.core;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Base interface for gathering and reporting statistics about a block of
+ * execution.
+ *
+ * <p>Spans should form a directed acyclic graph structure.  It should be
+ * possible to keep following the parents of a span until you arrive at a
+ * span with no parents.</p>
+ *
+ * * removed Jackson stuff, including annotation on Span and a serialization
+ *   implementation.
+ */
+public interface Span {
+  /**
+   * The block has completed, stop the clock
+   */
+  void stop();
+
+  /**
+   * Get the span start time.
+   *
+   * @return    The start time, in approximate milliseconds since the epoch.
+   */
+  long getStartTimeMillis();
+
+  /**
+   * Get the span stop time.
+   *
+   * @return    The stop time, in approximate milliseconds since the epoch.
+   */
+  long getStopTimeMillis();
+
+  /**
+   * Return the total amount of time elapsed since start was called, if running,
+   * or difference between stop and start
+   *
+   * @return    The elapsed time in milliseconds.
+   */
+  long getAccumulatedMillis();
+
+  /**
+   * Has the span been started and not yet stopped?
+   *
+   * @return    True if the span is still running (has no stop time).
+   */
+  boolean isRunning();
+
+  /**
+   * Return a textual description of this span.
+   *
+   * @return    The description of this span.  Will never be null.
+   */
+  String getDescription();
+
+  /**
+   * A pseudo-unique (random) number assigned to this span instance.
+   *
+   * @return    The spanID.  This object is immutable and is safe to access
+   *            from multiple threads.
+   */
+  SpanId getSpanId();
+
+  /**
+   * Create a child span of this span with the given description
+   * @deprecated Since 4.0.0. Use {@link MilliSpan.Builder}
+   * @param description The description to set on the child span.
+   * @return A new child span.
+   */
+  @Deprecated
+  Span child(String description);
+
+  @Override
+  String toString();
+
+  /**
+   * Returns the parent IDs of the span.
+   *
+   * @return The array of parents, or an empty array if there are no parents.
+   */
+  SpanId[] getParents();
+
+  /**
+   * Set the parents of this span.
+   *
+   * <p>Any existing parents will be cleared by this call.</p>
+   *
+   * @param parents     The parents to set.
+   */
+  void setParents(SpanId[] parents);
+
+  /**
+   * Add a data annotation associated with this span
+   *
+   * @param key         The key to set.
+   * @param value       The value to set.
+   */
+  void addKVAnnotation(String key, String value);
+
+  /**
+   * Add a timeline annotation associated with this span
+   *
+   * @param msg         The annotation to set.  It will be associated with
+   *                    the current time.
+   */
+  void addTimelineAnnotation(String msg);
+
+  /**
+   * Get the key-value annotations associated with this span.
+   *
+   * @return            The annotation map in read-only form.
+   *                    Will never be null.
+   */
+  Map<String, String> getKVAnnotations();
+
+  /**
+   * Get the timeline annotation list.
+   *
+   * @return            The annotation list in read-only form.
+   *                    Will never be null.
+   */
+  List<TimelineAnnotation> getTimelineAnnotations();
+
+  /**
+   * Return a unique id for the process from which this Span originated.
+   *
+   * @return            The tracer id.  Will never be null.
+   */
+  String getTracerId();
+
+  /**
+   * Set the tracer id of a span.
+   *
+   * @param s           The tracer ID to set.
+   */
+  void setTracerId(String s);
+
+  /**
+   * Serialize to Json
+   *
+   * @return            A JSON string with the span data.
+   */
+  String toJson();
+
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/SpanId.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/SpanId.java
new file mode 100644
index 0000000..f2e3413
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/SpanId.java
@@ -0,0 +1,89 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * Uniquely identifies an HTrace span.
+ *
+ * Span IDs are 128 bits in total.  The upper 64 bits of a span ID is the same
+ * as the upper 64 bits of the parent span, if there is one.  The lower 64 bits
+ * are always random.
+ *
+ * * All Span IDs are 0.
+ */
+public final class SpanId implements Comparable<SpanId> {
+
+  /**
+   * The invalid span ID, which is all zeroes.
+   *
+   * It is also the "least" span ID in the sense that it is considered
+   * smaller than any other span ID.
+   */
+  @SuppressWarnings("VisibilityModifier")
+  public static SpanId INVALID = new SpanId(0, 0);
+
+
+  public static SpanId fromRandom() {
+    return INVALID;
+  }
+
+  public static SpanId fromString(String str) {
+    return INVALID;
+  }
+
+  public SpanId(long high, long low) {
+  }
+
+  public long getHigh() {
+    return 0L;
+  }
+
+  public long getLow() {
+    return 0L;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof SpanId)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public int compareTo(SpanId other) {
+    return 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return 0;
+  }
+
+  @Override
+  public String toString() {
+    return "00000000000000000000000000000000";
+  }
+
+  public boolean isValid() {
+    return false;
+  }
+
+  public SpanId newChildId() {
+    return INVALID;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/SpanReceiver.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/SpanReceiver.java
new file mode 100644
index 0000000..4eefae8
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/SpanReceiver.java
@@ -0,0 +1,102 @@
+/*
+ * 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.htrace.core;
+
+import java.io.Closeable;
+
+
+/**
+ * The collector within a process that is the destination of Spans when a
+ * trace is running. {@code SpanReceiver} implementations are expected to
+ * provide a constructor with the signature
+ *
+ * <pre>
+ * <code>public SpanReceiverImpl(HTraceConfiguration)</code>
+ * </pre>
+ */
+public abstract class SpanReceiver implements Closeable {
+  /**
+   * A {@link SpanReceiver} builder. It takes a {@link SpanReceiver} class name
+   * and constructs an instance of that class, with the provided configuration.
+   */
+  public static class Builder {
+
+    public Builder(HTraceConfiguration conf) {
+    }
+
+    /**
+     * Set this builder back to defaults.
+     *
+     * @return this instance.
+     */
+    public Builder reset() {
+      return this;
+    }
+
+    public Builder className(final String className) {
+      return this;
+    }
+
+    /**
+     * Configure whether we should log errors during build().
+     *
+     * @param logErrors Whether we should log errors during build().
+     * @return This instance
+     */
+    public Builder logErrors(boolean logErrors) {
+      return this;
+    }
+
+    public Builder classLoader(ClassLoader classLoader) {
+      return this;
+    }
+
+    public SpanReceiver build() {
+      return Holder.SINGLETON;
+    }
+  }
+
+  /**
+   * Get the ID for this SpanReceiver.
+   *
+   * @return The unique ID identifying this SpanReceiver.
+   */
+  public final long getId() {
+    return 0xdeadbeefL;
+  }
+
+  protected SpanReceiver() {
+  }
+
+  /**
+   * Called when a Span is stopped and can now be stored.
+   *
+   * @param span    The span to store with this SpanReceiver.
+   */
+  public abstract void receiveSpan(Span span);
+
+  private static final class Holder {
+    static final SpanReceiver SINGLETON = new SpanReceiver() {
+      @Override
+      public void receiveSpan(Span span) {
+      }
+      @Override
+      public void close() {
+      }
+    };
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java
new file mode 100644
index 0000000..5ffb232
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java
@@ -0,0 +1,36 @@
+/*
+ * 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.htrace.core;
+
+import java.io.IOException;
+
+/**
+ * Used for testing. Simply prints to standard out any spans it receives.
+ */
+public class StandardOutSpanReceiver extends SpanReceiver {
+
+  public StandardOutSpanReceiver(HTraceConfiguration conf) {
+  }
+
+  @Override
+  public void receiveSpan(Span span) {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TimelineAnnotation.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TimelineAnnotation.java
new file mode 100644
index 0000000..8825e01
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TimelineAnnotation.java
@@ -0,0 +1,36 @@
+/*
+ * 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.htrace.core;
+
+public class TimelineAnnotation {
+
+  public TimelineAnnotation(long time, String msg) {
+  }
+
+  public long getTime() {
+    return 0;
+  }
+
+  public String getMessage() {
+    return "";
+  }
+
+  @Override
+  public String toString() {
+    return "";
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceCallable.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceCallable.java
new file mode 100644
index 0000000..13393c3
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceCallable.java
@@ -0,0 +1,40 @@
+/*
+ * 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.htrace.core;
+
+import java.util.concurrent.Callable;
+
+/**
+ * Wrap a Callable with a Span that survives a change in threads.
+ */
+public class TraceCallable<V> implements Callable<V> {
+  private final Callable<V> impl;
+
+  public TraceCallable(Tracer tracer, SpanId parentId, Callable<V> impl,
+      String description) {
+    this.impl = impl;
+  }
+
+  @Override
+  public V call() throws Exception {
+    return impl.call();
+  }
+
+  public Callable<V> getImpl() {
+    return impl;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceExecutorService.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceExecutorService.java
new file mode 100644
index 0000000..6aabf66
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceExecutorService.java
@@ -0,0 +1,111 @@
+/*
+ * 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.htrace.core;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A convenience wrapper around an {@link ExecutorService} for automatically
+ * propagating trace scopes to executable tasks.
+ */
+public class TraceExecutorService implements ExecutorService {
+  private final ExecutorService impl;
+
+  TraceExecutorService(Tracer tracer, String scopeName,
+                       ExecutorService impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public void execute(Runnable command) {
+    impl.execute(command);
+  }
+
+  @Override
+  public void shutdown() {
+    impl.shutdown();
+  }
+
+  @Override
+  public List<Runnable> shutdownNow() {
+    return impl.shutdownNow();
+  }
+
+  @Override
+  public boolean isShutdown() {
+    return impl.isShutdown();
+  }
+
+  @Override
+  public boolean isTerminated() {
+    return impl.isTerminated();
+  }
+
+  @Override
+  public boolean awaitTermination(long timeout, TimeUnit unit)
+      throws InterruptedException {
+    return impl.awaitTermination(timeout, unit);
+  }
+
+  @Override
+  public <T> Future<T> submit(Callable<T> task) {
+    return impl.submit(task);
+  }
+
+  @Override
+  public <T> Future<T> submit(Runnable task, T result) {
+    return impl.submit(task, result);
+  }
+
+  @Override
+  public Future<?> submit(Runnable task) {
+    return impl.submit(task);
+  }
+
+  @Override
+  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+      throws InterruptedException {
+    return impl.invokeAll(tasks);
+  }
+
+  @Override
+  public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks,
+                                       long timeout, TimeUnit unit) throws InterruptedException {
+    return impl.invokeAll(tasks, timeout, unit);
+  }
+
+  @Override
+  public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
+      throws InterruptedException, ExecutionException {
+    return impl.invokeAny(tasks);
+  }
+
+  @Override
+  public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout,
+                         TimeUnit unit) throws InterruptedException, ExecutionException,
+      TimeoutException {
+    return impl.invokeAny(tasks, timeout, unit);
+  }
+
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceRunnable.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceRunnable.java
new file mode 100644
index 0000000..738f7b1
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceRunnable.java
@@ -0,0 +1,51 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * Wrap a Runnable with a Span that survives a change in threads.
+ */
+public class TraceRunnable implements Runnable {
+  private final Runnable runnable;
+
+  /**
+   * @param tracer The Tracer to use for tracing.
+   * @param parent The TraceScope to read parent span ID from.
+   * @param runnable The Runnable that will be executed.
+   * @param description An optional description to set on the trace when executing.
+   * @deprecated Use {@link #TraceRunnable(Tracer, SpanId, Runnable, String)} instead.
+   */
+  @Deprecated
+  public TraceRunnable(Tracer tracer, TraceScope parent,
+      Runnable runnable, String description) {
+    this.runnable = runnable;
+  }
+
+  public TraceRunnable(Tracer tracer, SpanId parentId,
+      Runnable runnable, String description) {
+    this.runnable = runnable;
+  }
+
+  @Override
+  public void run() {
+    runnable.run();
+  }
+
+  public Runnable getRunnable() {
+    return runnable;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceScope.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceScope.java
new file mode 100644
index 0000000..b1bfaba
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TraceScope.java
@@ -0,0 +1,82 @@
+/*
+ * 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.htrace.core;
+
+import java.io.Closeable;
+
+/**
+ * Create a new TraceScope at major transitions. Hosts current tracing context.
+ */
+@SuppressWarnings("FinalClass")
+public class TraceScope implements Closeable {
+
+  private TraceScope() {
+  }
+
+  /**
+   * Returns the span which this scope is managing.
+   *
+   * @return The span.
+   */
+  public Span getSpan() {
+    return null;
+  }
+
+  /**
+   * Returns the span ID which this scope is managing.
+   *
+   * @return The span ID.
+   */
+  public SpanId getSpanId() {
+    return SpanId.INVALID;
+  }
+
+  /**
+   * Detach this TraceScope from the current thread.
+   *
+   * It is OK to "leak" TraceScopes which have been detached.  They will not
+   * consume any resources other than a small amount of memory until they are
+   * garbage collected.  On the other hand, trace scopes which are still
+   * attached must never be leaked.
+   */
+  public void detach() {
+  }
+
+  /**
+   * Attach this TraceScope to the current thread.
+   */
+  public void reattach() {
+  }
+
+  /**
+   * Close this TraceScope, ending the trace span it is managing.
+   */
+  @Override
+  public void close() {
+  }
+
+  public void addKVAnnotation(String key, String value) {
+  }
+
+  public void addTimelineAnnotation(String msg) {
+  }
+
+  static final class Holder {
+    public static final TraceScope SINGLETON = new TraceScope();
+  }
+
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Tracer.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Tracer.java
new file mode 100644
index 0000000..47ef23b
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/Tracer.java
@@ -0,0 +1,231 @@
+/*
+ * 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.htrace.core;
+
+import java.io.Closeable;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * Use a Tracer instance inside a 'process' to collect and distribute its trace Spans.
+ * Example processes are an HDFS DataNode or an HBase RegionServer. A Tracer instance is your
+ * one-stop shop for all things tracing.
+ *
+ */
+@SuppressWarnings("FinalClass")
+public class Tracer implements Closeable {
+  public final static String SPAN_RECEIVER_CLASSES_KEY = "span.receiver.classes";
+  public final static String SAMPLER_CLASSES_KEY = "sampler.classes";
+
+  public static class Builder {
+    /**
+     * @deprecated Since 4.0.0. Use Constructor that takes a <code>name</code> argument instead
+     */
+    @Deprecated
+    public Builder() {
+    }
+
+    public Builder(final String name) {
+    }
+
+    /**
+     * @param name The name of the Tracer to create.
+     * @return this
+     * @deprecated Since 4.0.0. Use Constructor that takes a <code>name</code> argument instead.
+     */
+    @Deprecated
+    public Builder name(String name) {
+      return this;
+    }
+
+    /**
+     * @param conf The configuration to set.
+     * @return this
+     */
+    public Builder conf(HTraceConfiguration conf) {
+      return this;
+    }
+
+    /**
+     * @param tracerPool The pool to set.
+     * @return this
+     */
+    public Builder tracerPool(TracerPool tracerPool) {
+      return this;
+    }
+
+    /**
+     * @return The new Tracer object.
+     */
+    public Tracer build() {
+      return SINGLETON;
+    }
+  }
+
+  private final static Tracer SINGLETON = new Tracer();
+
+  /**
+   * @return If the current thread is tracing, this function returns the Tracer that is
+   *   being used; otherwise, it returns null.
+   */
+  public static Tracer curThreadTracer() {
+    return SINGLETON;
+  }
+
+  private Tracer() {
+  }
+
+  public String getTracerId() {
+    return "";
+  }
+
+  /**
+   * Create a new trace scope.
+   *
+   * If there are no scopes above the current scope, we will apply our
+   * configured samplers. Otherwise, we will create a trace Span only if this thread
+   * is already tracing, or if the passed parentID was valid.
+   *
+   * @param description         The description of the new span to create.
+   * @param parentId            If this is a valid span ID, it will be added to
+   *                              the parents of the new span we create.
+   * @return                    The new trace scope.
+   */
+  public TraceScope newScope(String description, SpanId parentId) {
+    return TraceScope.Holder.SINGLETON;
+  }
+
+  /**
+   * Create a new trace scope.
+   *
+   * If there are no scopes above the current scope, we will apply our
+   * configured samplers. Otherwise, we will create a trace Span only if this thread
+   * is already tracing.
+   * @param description         The description of the new span to create.
+   * @return                    The new trace scope.
+   */
+  public TraceScope newScope(String description) {
+    return TraceScope.Holder.SINGLETON;
+  }
+
+  /**
+   * Return a null trace scope.
+   *
+   * @return The null trace scope.
+   */
+  public TraceScope newNullScope() {
+    return TraceScope.Holder.SINGLETON;
+  }
+
+  /**
+   * Wrap the callable in a TraceCallable, if tracing.
+   *
+   * @param <V>          The subclass of callable.
+   * @param callable     The callable to wrap.
+   * @param description  A description of the callable, or null if there
+   *                     is no description.
+   * @return The callable provided, wrapped if tracing, 'callable' if not.
+   */
+  public <V> Callable<V> wrap(Callable<V> callable, String description) {
+    return callable;
+  }
+
+  /**
+   * Wrap the runnable in a TraceRunnable, if tracing
+   *
+   * @param runnable      The runnable to wrap.
+   * @param description   A description of the runnable, or null if there is
+   *                      no description.
+   * @return The runnable provided, wrapped if tracing, 'runnable' if not.
+   */
+  public Runnable wrap(Runnable runnable, String description) {
+    return runnable;
+  }
+
+  public TraceExecutorService newTraceExecutorService(ExecutorService impl) {
+    return newTraceExecutorService(impl, null);
+  }
+
+  public TraceExecutorService newTraceExecutorService(ExecutorService impl,
+                                                      String scopeName) {
+    return new TraceExecutorService(this, scopeName, impl);
+  }
+
+  public ScheduledTraceExecutorService newTraceExecutorService(
+      ScheduledExecutorService impl) {
+    return newTraceExecutorService(impl, null);
+  }
+
+  public ScheduledTraceExecutorService newTraceExecutorService(
+      ScheduledExecutorService impl, String scopeName) {
+    return new ScheduledTraceExecutorService(this, scopeName, impl);
+  }
+
+  public TracerPool getTracerPool() {
+    return TracerPool.getGlobalTracerPool();
+  }
+
+  /**
+   * Returns an array of all the current Samplers.
+   *
+   * Note that if the current Samplers change, those changes will not be
+   * reflected in this array.  In other words, this array may be stale.
+   *
+   * @return The current samplers.
+   */
+  public Sampler[] getSamplers() {
+    return new Sampler[0];
+  }
+
+  /**
+   * Add a new Sampler.
+   *
+   * @param sampler       The new sampler to add.
+   *                      You cannot add a particular Sampler object more
+   *                        than once.  You may add multiple Sampler objects
+   *                        of the same type, although this is not recommended.
+   *
+   * @return              True if the sampler was added; false if it already had
+   *                        been added earlier.
+   */
+  public synchronized boolean addSampler(Sampler sampler) {
+    return true;
+  }
+
+  /**
+   * Remove a Sampler.
+   *
+   * @param sampler       The sampler to remove.
+   * @return              True only if the sampler was removed.
+   */
+  public synchronized boolean removeSampler(Sampler sampler) {
+    return false;
+  }
+
+  public static Span getCurrentSpan() {
+    return null;
+  }
+
+  public static SpanId getCurrentSpanId() {
+    return SpanId.INVALID;
+  }
+
+  @Override
+  public synchronized void close() {
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TracerId.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TracerId.java
new file mode 100644
index 0000000..abb2c96
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TracerId.java
@@ -0,0 +1,55 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * <p>The HTrace tracer ID.</p>
+ *
+ * <p>HTrace tracer IDs are created from format strings.
+ * Format strings contain variables which the TracerId class will
+ * replace with the correct values at runtime.</p>
+ *
+ * <ul>
+ * <li>%{tname}: the tracer name supplied when creating the Tracer.</li>
+ * <li>%{pname}: the process name obtained from the JVM.</li>
+ * <li>%{ip}: will be replaced with an ip address.</li>
+ * <li>%{pid}: the numerical process ID from the operating system.</li>
+ * </ul>
+ *
+ * <p>For example, the string "%{pname}/%{ip}" will be replaced with something
+ * like: DataNode/192.168.0.1, assuming that the process' name is DataNode
+ * and its IP address is 192.168.0.1.</p>
+ *
+ *  ID strings can contain backslashes as escapes.
+ * For example, "\a" will map to "a".  "\%{ip}" will map to the literal
+ * string "%{ip}", not the IP address.  A backslash itself can be escaped by a
+ * preceding backslash.
+ */
+public final class TracerId {
+
+  /**
+   * The configuration key to use for process id
+   */
+  public static final String TRACER_ID_KEY = "tracer.id";
+
+  public TracerId(HTraceConfiguration conf, String tracerName) {
+  }
+
+  public String get() {
+    return TRACER_ID_KEY;
+  }
+}
diff --git a/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TracerPool.java b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TracerPool.java
new file mode 100644
index 0000000..22c83bd
--- /dev/null
+++ b/hbase-noop-htrace/src/main/java/org/apache/htrace/core/TracerPool.java
@@ -0,0 +1,155 @@
+/*
+ * 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.htrace.core;
+
+/**
+ * A pool of Tracer objects.
+ *
+ * There may be more than one {@link Tracer} running inside a single 'process'; for example,
+ * unit tests may spin up a DataNode, a NameNode, and HDFS clients all running in a single JVM
+ * instance, each with its own Tracer. TracerPool is where all Tracer instances register
+ * on creation so Tracers can coordinate around shared resources such as {@link SpanReceiver}
+ * instances. TracerPool takes care of properly cleaning up registered Tracer instances on shutdown.
+ */
+public class TracerPool {
+
+  /**
+   * The global pool of tracer objects.
+   *
+   * This is the pool that new tracers get put into by default.
+   */
+  static final TracerPool GLOBAL = new TracerPool("Global");
+
+  /**
+   * Get the global tracer pool.
+   *
+   * @return The tracer pool.
+   */
+  public static TracerPool getGlobalTracerPool() {
+    return GLOBAL;
+  }
+
+  public TracerPool(String name) {
+  }
+
+  /**
+   * Return the name of this TracerPool.
+   *
+   * @return The name.
+   */
+  public String getName() {
+    return "";
+  }
+
+  /**
+   * Returns an array of all the current span receivers.
+   *
+   * Note that if the current span receivers change, those changes will not be
+   * reflected in this array.  In other words, this array may be stale.
+   *
+   * @return An array of the current span receivers.
+   */
+  public SpanReceiver[] getReceivers() {
+    return new SpanReceiver[0];
+  }
+
+  /**
+   * Add a new span receiver.
+   *
+   * @param receiver        The new receiver to add.
+   *
+   * @return                True if the new receiver was added; false if it
+   *                          already was there.
+   */
+  public synchronized boolean addReceiver(SpanReceiver receiver) {
+    // since we are not tracking these, maybe we should close them immediately?
+    // for now do nothing, since we should just be getting empty receivers that do not do anything.
+    return true;
+  }
+
+  /**
+   * Remove a span receiver.
+   *
+   * @param receiver        The receiver to remove.
+   *
+   * @return                True if the receiver was removed; false if it
+   *                          did not exist in this pool.
+   */
+  public synchronized boolean removeReceiver(SpanReceiver receiver) {
+    return false;
+  }
+
+  /**
+   * Remove and close a span receiver.
+   *
+   * @param receiver        The receiver to remove.
+   *
+   * @return                True if the receiver was removed; false if it
+   *                          did not exist in this pool.
+   */
+  public boolean removeAndCloseReceiver(SpanReceiver receiver) {
+    return false;
+  }
+
+  /**
+   * Given a SpanReceiver class name, return the existing instance of that span
+   * receiver, if possible; otherwise, invoke the callable to create a new
+   * instance.
+   *
+   * @param className       The span receiver class name.
+   * @param conf            The HTrace configuration.
+   * @param classLoader     The class loader to use.
+   *
+   * @return                The SpanReceiver.
+   */
+  public synchronized SpanReceiver loadReceiverType(String className,
+      HTraceConfiguration conf, ClassLoader classLoader) {
+    SpanReceiver receiver = new SpanReceiver.Builder(conf).
+        className(className).
+        classLoader(classLoader).
+        build();
+    return receiver;
+  }
+
+  /**
+   * Returns an array of all the current Tracers.
+   *
+   * Note that if the current Tracers change, those changes will not be
+   * reflected in this array.  In other words, this array may be stale.
+   *
+   * @return The current array of tracers.
+   */
+  public synchronized Tracer[] getTracers() {
+    return new Tracer[0];
+  }
+
+  /**
+   * Add a new Tracer.
+   */
+  synchronized void addTracer(Tracer tracer) {
+  }
+
+  /**
+   * Remove a Tracer.
+   *
+   * If the Tracer removed was the last one, we will close all the SpanReceiver
+   * objects that we're managing.
+   */
+  synchronized void removeTracer(Tracer tracer) {
+  }
+
+}
diff --git a/hbase-noop-htrace/src/test/java/org/apache/hbase/htrace_noop/TestHBaseTestingUtilityWithHTraceNoop.java b/hbase-noop-htrace/src/test/java/org/apache/hbase/htrace_noop/TestHBaseTestingUtilityWithHTraceNoop.java
new file mode 100644
index 0000000..507e72a
--- /dev/null
+++ b/hbase-noop-htrace/src/test/java/org/apache/hbase/htrace_noop/TestHBaseTestingUtilityWithHTraceNoop.java
@@ -0,0 +1,62 @@
+/**
+ *
+ * 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.hbase.htrace_noop;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestHBaseTestingUtilityWithHTraceNoop {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testPutThenCountWithNewTable() throws Exception {
+    TableName tableName = TableName.valueOf("test");
+
+    Table table = TEST_UTIL.createTable(tableName, "cf");
+
+    Put put1 = new Put(Bytes.toBytes("r1"));
+    put1.addColumn(Bytes.toBytes("cf"), Bytes.toBytes("c"), Bytes.toBytes(1));
+    table.put(put1);
+
+    Put put2 = new Put(Bytes.toBytes("r2"));
+    put2.addColumn(Bytes.toBytes("cf"), Bytes.toBytes("c"), Bytes.toBytes(2));
+    table.put(put2);
+
+    int rows = TEST_UTIL.countRows(tableName);
+    assertEquals(2, rows);
+  }
+}
diff --git a/pom.xml b/pom.xml
index 6e3a136..ef5af83 100644
--- a/pom.xml
+++ b/pom.xml
@@ -59,6 +59,7 @@
     <module>hbase-shaded-miscellaneous</module>
     <module>hbase-shaded-jetty</module>
     <module>hbase-shaded-jersey</module>
+    <module>hbase-noop-htrace</module>
   </modules>
   <scm>
     <connection>scm:git:git://git.apache.org/hbase-thirdparty.git</connection>