You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by cr...@apache.org on 2014/07/30 00:01:11 UTC

git commit: SAMZA-202; support java 8 for runtime and compile time

Repository: incubator-samza
Updated Branches:
  refs/heads/master da79b6f92 -> 3a8e2f9d1


SAMZA-202; support java 8 for runtime and compile time


Project: http://git-wip-us.apache.org/repos/asf/incubator-samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-samza/commit/3a8e2f9d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-samza/tree/3a8e2f9d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-samza/diff/3a8e2f9d

Branch: refs/heads/master
Commit: 3a8e2f9d1d126cff84b791dd11d624bba8d5c80b
Parents: da79b6f
Author: Chris Riccomini <cr...@criccomi-mn.linkedin.biz>
Authored: Tue Jul 29 15:01:04 2014 -0700
Committer: Chris Riccomini <cr...@criccomi-mn.linkedin.biz>
Committed: Tue Jul 29 15:01:04 2014 -0700

----------------------------------------------------------------------
 bin/check-all.sh                                | 69 ++++++++++++++++++++
 bin/generate-javadocs.sh                        | 38 +++++++++++
 bin/publish-site.sh                             | 51 +++++++++++++++
 build.gradle                                    |  2 +
 docs/README.md                                  |  4 +-
 docs/_tools/generate-javadocs.sh                | 38 -----------
 docs/_tools/publish-site.sh                     | 51 ---------------
 gradle.properties                               |  2 +-
 gradle/dependency-versions-scala-2.10.gradle    |  2 +-
 gradle/maven.gradle                             |  4 +-
 gradle/wrapper/gradle-wrapper.properties        |  4 +-
 .../org/apache/samza/storage/StorageEngine.java | 13 ++--
 .../org/apache/samza/system/SystemConsumer.java | 10 +--
 .../samza/task/TaskLifecycleListener.java       | 34 ++++++++++
 .../apache/samza/storage/kv/KeyValueStore.java  |  4 +-
 15 files changed, 219 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/bin/check-all.sh
----------------------------------------------------------------------
diff --git a/bin/check-all.sh b/bin/check-all.sh
new file mode 100755
index 0000000..cc31942
--- /dev/null
+++ b/bin/check-all.sh
@@ -0,0 +1,69 @@
+#!/bin/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.
+
+# Run the gradlew check task against all supported JDKs and Scala versions.
+# Requires appropriate JAVAx_HOME variables to be set for Java 6, 7 and 8.
+
+set -e
+
+SCALAs=( "2.9.2" "2.10" )
+JDKs=( "JAVA6_HOME" "JAVA7_HOME" "JAVA8_HOME" )
+
+# get base directory
+home_dir=`pwd`
+base_dir=$(dirname $0)/..
+cd $base_dir
+base_dir=`pwd`
+cd $home_dir
+
+# validate JDKs, and default when appropriate
+any_not_set=false
+for i in "${JDKs[@]}"
+do
+  if [ -z "${!i}" ]; then
+    echo "${i} is not set."
+
+    # if java_home is available (usually OSX), then use it to find JAVA_HOME
+    if [ -f /usr/libexec/java_home ]; then
+      jdk_number=${i:4:1}
+      jdk_version=$(/usr/libexec/java_home -v 1.${jdk_number})
+      eval "${i}=${jdk_version}";
+      echo "${i} defaulted to ${!i}"
+    else
+      any_not_set=true
+    fi
+  fi
+done
+
+if [ "$any_not_set" = "true" ]; then
+  echo "Be sure that all necessary JAVA home variables are set and re-run."
+  exit 0
+fi
+
+# run all checks
+for i in "${JDKs[@]}"
+do
+  for scala_version in "${SCALAs[@]}"
+  do
+    jdk_number=${i:4:1}
+    echo "------------- Running check task against JDK${jdk_number}/Scala ${scala_version}"
+    JAVA_HOME=${!i}
+    $base_dir/gradlew -PscalaVersion=${scala_version} clean check
+    echo "------------- Finished running check task against JDK${jdk_number}/Scala ${scala_version}"
+  done
+done

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/bin/generate-javadocs.sh
----------------------------------------------------------------------
diff --git a/bin/generate-javadocs.sh b/bin/generate-javadocs.sh
new file mode 100755
index 0000000..112684f
--- /dev/null
+++ b/bin/generate-javadocs.sh
@@ -0,0 +1,38 @@
+#!/bin/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.
+
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+BASE_DIR=$DIR/..
+VERSION=$1
+JAVADOC_DIR=$BASE_DIR/docs/learn/documentation/$VERSION/api/javadocs
+
+if test -z "$VERSION"; then
+  echo
+  echo "  USAGE:"
+  echo
+  echo "    ${BASH_SOURCE[0]##*/} 0.7.0"
+  echo
+  exit 0
+fi
+
+cd $BASE_DIR
+./gradlew javadoc
+rm -rf $JAVADOC_DIR
+mkdir -p $JAVADOC_DIR
+cp -r $BASE_DIR/samza-api/build/docs/javadoc/* $JAVADOC_DIR
+cd -

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/bin/publish-site.sh
----------------------------------------------------------------------
diff --git a/bin/publish-site.sh b/bin/publish-site.sh
new file mode 100755
index 0000000..8da57a9
--- /dev/null
+++ b/bin/publish-site.sh
@@ -0,0 +1,51 @@
+#!/bin/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.
+
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+BASE_DIR=$DIR/..
+DOCS_DIR=$BASE_DIR/docs
+VERSION=$1
+COMMENT=$2
+USER=$3
+
+if test -z "$VERSION" || test -z "$COMMENT" || test -z "$USER"; then
+  echo
+  echo "  USAGE:"
+  echo
+  echo "    ${BASH_SOURCE[0]##*/} 0.7.0 \"updating welcome page\" criccomini"
+  echo
+  exit 0
+fi
+
+echo "Using uer: $USER"
+echo "Using version: $VERSION"
+echo "Using comment: $COMMENT"
+echo "Generating javadocs."
+$BASE_DIR/bin/generate-javadocs.sh $VERSION
+
+echo "Building site."
+cd $DOCS_DIR
+bundle exec jekyll build
+
+echo "Checking out SVN site."
+SVN_TMP=`mktemp -d /tmp/samza-svn.XXXX`
+svn co https://svn.apache.org/repos/asf/incubator/samza/ $SVN_TMP
+cp -r _site/* $SVN_TMP/site/
+svn add --force $SVN_TMP/site
+svn commit $SVN_TMP -m"$COMMENT" --username $USER
+rm -rf $SVN_TMP

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index c262b5f..3ad5fe3 100644
--- a/build.gradle
+++ b/build.gradle
@@ -192,9 +192,11 @@ project(":samza-yarn_$scalaVersion") {
     }
     compile("org.apache.hadoop:hadoop-yarn-common:$yarnVersion") {
       exclude module: 'slf4j-log4j12'
+      exclude module: 'servlet-api'
     }
     compile("org.apache.hadoop:hadoop-yarn-client:$yarnVersion") {
       exclude module: 'slf4j-log4j12'
+      exclude module: 'servlet-api'
     }
     compile("org.apache.hadoop:hadoop-common:$yarnVersion") {
       exclude module: 'slf4j-log4j12'

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/docs/README.md
----------------------------------------------------------------------
diff --git a/docs/README.md b/docs/README.md
index 955280e..de5fc23 100644
--- a/docs/README.md
+++ b/docs/README.md
@@ -40,7 +40,7 @@ All links between pages inside a versioned folder should be relative links, not
 
 To auto-generate the latest Javadocs, run:
 
-    _tools/generate-javadocs.sh <version>
+    bin/generate-javadocs.sh <version>
 
 The version number is the number that will be used in the /docs/learn/documentation/<version>/api/javadocs path.
 
@@ -48,6 +48,6 @@ The version number is the number that will be used in the /docs/learn/documentat
 
 To build and publish the website to Samza's Apache SVN repository, run:
 
-    _tools/publish-site.sh 0.7.0 "updating welcome page" criccomini
+    bin/publish-site.sh 0.7.0 "updating welcome page" criccomini
 
 This command will re-build the Javadocs and website, checkout https://svn.apache.org/repos/asf/incubator/samza/site/ locally, copy the site into the directory, and commit the changes.

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/docs/_tools/generate-javadocs.sh
----------------------------------------------------------------------
diff --git a/docs/_tools/generate-javadocs.sh b/docs/_tools/generate-javadocs.sh
deleted file mode 100755
index 0a09a66..0000000
--- a/docs/_tools/generate-javadocs.sh
+++ /dev/null
@@ -1,38 +0,0 @@
-#!/bin/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.
-
-DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
-BASE_DIR=$DIR/../..
-VERSION=$1
-JAVADOC_DIR=$BASE_DIR/docs/learn/documentation/$VERSION/api/javadocs
-
-if test -z "$VERSION"; then
-  echo
-  echo "  USAGE:"
-  echo
-  echo "    ${BASH_SOURCE[0]##*/} 0.7.0"
-  echo
-  exit 0
-fi
-
-cd $BASE_DIR
-./gradlew javadoc
-rm -rf $JAVADOC_DIR
-mkdir -p $JAVADOC_DIR
-cp -r $BASE_DIR/samza-api/build/docs/javadoc/* $JAVADOC_DIR
-cd -

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/docs/_tools/publish-site.sh
----------------------------------------------------------------------
diff --git a/docs/_tools/publish-site.sh b/docs/_tools/publish-site.sh
deleted file mode 100755
index a65e760..0000000
--- a/docs/_tools/publish-site.sh
+++ /dev/null
@@ -1,51 +0,0 @@
-#!/bin/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.
-
-DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
-BASE_DIR=$DIR/../..
-DOCS_DIR=$BASE_DIR/docs
-VERSION=$1
-COMMENT=$2
-USER=$3
-
-if test -z "$VERSION" || test -z "$COMMENT" || test -z "$USER"; then
-  echo
-  echo "  USAGE:"
-  echo
-  echo "    ${BASH_SOURCE[0]##*/} 0.7.0 \"updating welcome page\" criccomini"
-  echo
-  exit 0
-fi
-
-echo "Using uer: $USER"
-echo "Using version: $VERSION"
-echo "Using comment: $COMMENT"
-echo "Generating javadocs."
-$DOCS_DIR/_tools/generate-javadocs.sh $VERSION
-
-echo "Building site."
-cd $DOCS_DIR
-bundle exec jekyll build
-
-echo "Checking out SVN site."
-SVN_TMP=`mktemp -d /tmp/samza-svn.XXXX`
-svn co https://svn.apache.org/repos/asf/incubator/samza/ $SVN_TMP
-cp -r _site/* $SVN_TMP/site/
-svn add --force $SVN_TMP/site
-svn commit $SVN_TMP -m"$COMMENT" --username $USER
-rm -rf $SVN_TMP

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/gradle.properties
----------------------------------------------------------------------
diff --git a/gradle.properties b/gradle.properties
index 0a5bde5..6f35fb4 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -18,6 +18,6 @@ group=org.apache.samza
 version=0.8.0-SNAPSHOT
 scalaVersion=2.10
 
-gradleVersion=1.6
+gradleVersion=2.0
 
 org.gradle.jvmargs="-XX:MaxPermSize=512m"

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/gradle/dependency-versions-scala-2.10.gradle
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions-scala-2.10.gradle b/gradle/dependency-versions-scala-2.10.gradle
index 7094d6c..ba54099 100644
--- a/gradle/dependency-versions-scala-2.10.gradle
+++ b/gradle/dependency-versions-scala-2.10.gradle
@@ -18,7 +18,7 @@
  */
 ext {
   scalaVersion = "2.10"
-  scalaLibVersion = "2.10.2"
+  scalaLibVersion = "2.10.4"
   // Extra options for the compiler:
   // -feature: Give detailed warnings about language feature use (rather than just 'there were 4 warnings')
   // -language:implicitConversions: Allow the use of implicit conversions without warning or library import

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/gradle/maven.gradle
----------------------------------------------------------------------
diff --git a/gradle/maven.gradle b/gradle/maven.gradle
index a1f62d0..7a4eb71 100644
--- a/gradle/maven.gradle
+++ b/gradle/maven.gradle
@@ -23,14 +23,12 @@ subprojects {
     publications {
       mavenJava(MavenPublication) {
         from components.java
-
         if(project.name == 'samza-shell') {
           artifact shellTarGz {
-            name "samza-shell"
             classifier "dist"
           }
         }
-      }
+     }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/gradle/wrapper/gradle-wrapper.properties
----------------------------------------------------------------------
diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties
index 666eff6..78596c0 100644
--- a/gradle/wrapper/gradle-wrapper.properties
+++ b/gradle/wrapper/gradle-wrapper.properties
@@ -1,6 +1,6 @@
-#Fri Jun 13 10:45:15 PDT 2014
+#Thu Jul 03 20:51:36 PDT 2014
 distributionBase=GRADLE_USER_HOME
 distributionPath=wrapper/dists
 zipStoreBase=GRADLE_USER_HOME
 zipStorePath=wrapper/dists
-distributionUrl=http\://services.gradle.org/distributions/gradle-1.6-bin.zip
+distributionUrl=http\://services.gradle.org/distributions/gradle-2.0-bin.zip

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java
index 747ee2b..5463648 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java
+++ b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java
@@ -26,18 +26,23 @@ import org.apache.samza.system.IncomingMessageEnvelope;
 /**
  * A storage engine for managing state maintained by a stream processor.
  * 
- * <p>This interface does not specify any query capabilities, which, of course,
+ * <p>
+ * This interface does not specify any query capabilities, which, of course,
  * would be query engine specific. Instead it just specifies the minimum
  * functionality required to reload a storage engine from its changelog as well
  * as basic lifecycle management.
+ * </p>
  */
 public interface StorageEngine {
 
   /**
-   * Restore the content of this StorageEngine from the changelog.  Messages are provided
-   * in one {@link java.util.Iterator} and not deserialized for efficiency, allowing the
-   * implementation to optimize replay, if possible.
+   * Restore the content of this StorageEngine from the changelog. Messages are
+   * provided in one {@link java.util.Iterator} and not deserialized for
+   * efficiency, allowing the implementation to optimize replay, if possible.
+   * 
    * @param envelopes
+   *          An iterator of envelopes that the storage engine can read from to
+   *          restore its state on startup.
    */
   void restore(Iterator<IncomingMessageEnvelope> envelopes);
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java b/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java
index 37c6c76..8dfcc74 100644
--- a/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemConsumer.java
@@ -47,10 +47,10 @@ import java.util.Set;
  * <li>Start will be called before stop.</li>
  * <li>Register will be called one or more times before start.</li>
  * <li>Register won't be called twice for the same SystemStreamPartition.</li>
- * <li>If timeout < 0, poll will block unless all SystemStreamPartition are at
+ * <li>If timeout &lt; 0, poll will block unless all SystemStreamPartition are at
  * "head" (the underlying system has been checked, and returned an empty set).
  * If at head, an empty list is returned.</li>
- * <li>If timeout >= 0, poll will return any messages that are currently
+ * <li>If timeout &gt;= 0, poll will return any messages that are currently
  * available for any of the SystemStreamPartitions specified. If no new messages
  * are available, it will wait up to timeout milliseconds for messages from any
  * SystemStreamPartition to become available. It will return an empty list if
@@ -154,10 +154,10 @@ public interface SystemConsumer {
    *          be returned. It is illegal to pass in SystemStreamPartitions that
    *          have not been registered with the SystemConsumer first.
    * @param timeout
-   *          If timeout < 0, poll will block unless all SystemStreamPartition
+   *          If timeout &lt; 0, poll will block unless all SystemStreamPartition
    *          are at "head" (the underlying system has been checked, and
    *          returned an empty set). If at head, an empty map is returned. If
-   *          timeout >= 0, poll will return any messages that are currently
+   *          timeout &gt;= 0, poll will return any messages that are currently
    *          available for any of the SystemStreamPartitions specified. If no
    *          new messages are available, it will wait up to timeout
    *          milliseconds for messages from any SystemStreamPartition to become
@@ -171,6 +171,8 @@ public interface SystemConsumer {
    *         new messages are available for any SystemStreamPartitions in the
    *         input set.
    * @throws InterruptedException
+   *          Thrown when a blocking poll has been interrupted by another
+   *          thread.
    */
   Map<SystemStreamPartition, List<IncomingMessageEnvelope>> poll(Set<SystemStreamPartition> systemStreamPartitions, long timeout) throws InterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java b/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java
index fa2b52d..55524a1 100644
--- a/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java
+++ b/samza-api/src/main/java/org/apache/samza/task/TaskLifecycleListener.java
@@ -29,31 +29,65 @@ import org.apache.samza.system.IncomingMessageEnvelope;
 public interface TaskLifecycleListener {
   /**
    * Called before all tasks in TaskRunner are initialized.
+   * 
+   * @param config
+   *          Config for the Samza job.
+   * @param context
+   *          TaskContext for the StreamTask that's being initialized.
    */
   void beforeInit(Config config, TaskContext context);
 
   /**
    * Called after all tasks in TaskRunner are initialized.
+   * 
+   * @param config
+   *          Config for the Samza job.
+   * @param context
+   *          TaskContext for the StreamTask that's being initialized.
    */
   void afterInit(Config config, TaskContext context);
 
   /**
    * Called before a message is processed by a task.
+   * 
+   * @param envelope
+   *          The envelope to be processed by the StreamTask.
+   * @param config
+   *          Config for the Samza job.
+   * @param context
+   *          TaskContext for the StreamTask that's about to process a message.
    */
   void beforeProcess(IncomingMessageEnvelope envelope, Config config, TaskContext context);
 
   /**
    * Called after a message is processed by a task.
+   * 
+   * @param envelope
+   *          The envelope that was processed by the StreamTask.
+   * @param config
+   *          Config for the Samza job.
+   * @param context
+   *          TaskContext for the StreamTask that just processed a message.
    */
   void afterProcess(IncomingMessageEnvelope envelope, Config config, TaskContext context);
 
   /**
    * Called before all tasks in TaskRunner are closed.
+   * 
+   * @param config
+   *          Config for the Samza job.
+   * @param context
+   *          TaskContext for the StreamTask that's about to be shutdown.
    */
   void beforeClose(Config config, TaskContext context);
 
   /**
    * Called after all tasks in TaskRunner are closed.
+   * 
+   * @param config
+   *          Config for the Samza job.
+   * @param context
+   *          TaskContext for the StreamTask that was just shutdown.
    */
   void afterClose(Config config, TaskContext context);
 }

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/3a8e2f9d/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java
index bdaa234..b708341 100644
--- a/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java
@@ -47,6 +47,7 @@ public interface KeyValueStore<K, V> {
   
   /**
    * Update all the given key/value pairs
+   * @param entries A list of entries to put into the store.
    * @throws NullPointerException If null is used for any key or value.
    */
   public void putAll(List<Entry<K,V>> entries);
@@ -68,7 +69,8 @@ public interface KeyValueStore<K, V> {
   public KeyValueIterator<K,V> range(K from, K to);
   
   /**
-   * Return an iterator over all keys in the database. This iterator MUST be closed after use.
+   * Return an iterator over all keys in the database. This iterator MUST be closed after use. 
+   * @return An iterator of all key/value pairs in the store.
    */
   public KeyValueIterator<K,V> all();