You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2012/01/12 20:03:08 UTC
svn commit: r1230706 -
/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
Author: vines
Date: Thu Jan 12 19:03:07 2012
New Revision: 1230706
URL: http://svn.apache.org/viewvc?rev=1230706&view=rev
Log:
Fixes ACCUMULO-310 - This also needs to be done with FileOutputFormat
Modified:
incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java?rev=1230706&r1=1230705&r2=1230706&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java (original)
+++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java Thu Jan 12 19:03:07 2012
@@ -87,28 +87,48 @@ public class AccumuloFileOutputFormat ex
};
}
+ /**
+ * @deprecated Use {@link #handleBlockSize(Configuration)} instead
+ */
protected static void handleBlockSize(JobContext job) {
- Configuration conf = job.getConfiguration();
+ handleBlockSize(job.getConfiguration());
+ }
+
+ protected static void handleBlockSize(Configuration conf) {
int blockSize;
if (conf.getBoolean(INSTANCE_HAS_BEEN_SET, false)) {
- blockSize = (int) getInstance(job).getConfiguration().getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE);
+ blockSize = (int) getInstance(conf).getConfiguration().getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE);
} else {
- blockSize = getBlockSize(job);
+ blockSize = getBlockSize(conf);
}
conf.setInt("io.seqfile.compress.blocksize", blockSize);
}
+ /**
+ * @deprecated Use {@link #setFileType(Configuration,String)} instead
+ */
public static void setFileType(JobContext job, String type) {
- job.getConfiguration().set(FILE_TYPE, type);
+ setFileType(job.getConfiguration(), type);
+ }
+
+ public static void setFileType(Configuration conf, String type) {
+ conf.set(FILE_TYPE, type);
}
+ /**
+ * @deprecated Use {@link #setBlockSize(Configuration,int)} instead
+ */
public static void setBlockSize(JobContext job, int blockSize) {
- job.getConfiguration().setInt(BLOCK_SIZE, blockSize);
+ setBlockSize(job.getConfiguration(), blockSize);
+ }
+
+ public static void setBlockSize(Configuration conf, int blockSize) {
+ conf.setInt(BLOCK_SIZE, blockSize);
}
- private static int getBlockSize(JobContext job) {
- return job.getConfiguration().getInt(BLOCK_SIZE,
+ private static int getBlockSize(Configuration conf) {
+ return conf.getInt(BLOCK_SIZE,
(int) AccumuloConfiguration.getDefaultConfiguration().getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
}
@@ -117,10 +137,20 @@ public class AccumuloFileOutputFormat ex
* @param job
* @param instanceName
* @param zooKeepers
- * @deprecated since 1.4, use {@link #setBlockSize(JobContext, int)} to set block size no other configurations are used by OutputFormat
+ * @deprecated since 1.4, use {@link #setBlockSize(Configuration, int)} to set block size no other configurations are used by OutputFormat
*/
public static void setZooKeeperInstance(JobContext job, String instanceName, String zooKeepers) {
- Configuration conf = job.getConfiguration();
+ setZooKeeperInstance(job.getConfiguration(), instanceName, zooKeepers);
+ }
+
+ /**
+ *
+ * @param conf
+ * @param instanceName
+ * @param zooKeepers
+ * @deprecated since 1.4, use {@link #setBlockSize(Configuration, int)} to set block size no other configurations are used by OutputFormat
+ */
+ public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
if (conf.getBoolean(INSTANCE_HAS_BEEN_SET, false))
throw new IllegalStateException("Instance info can only be set once per job");
conf.setBoolean(INSTANCE_HAS_BEEN_SET, true);
@@ -135,10 +165,20 @@ public class AccumuloFileOutputFormat ex
* @param job
* @return
*
- * @deprecated since 1.4, use {@link #setBlockSize(JobContext, int)} no other configurations are used by OutputFormat
+ * @deprecated since 1.4, use {@link #setBlockSize(Configuration, int)} no other configurations are used by OutputFormat
*/
protected static Instance getInstance(JobContext job) {
- Configuration conf = job.getConfiguration();
+ return getInstance(job.getConfiguration());
+ }
+
+ /**
+ *
+ * @param conf
+ * @return
+ *
+ * @deprecated since 1.4, use {@link #setBlockSize(Configuration, int)} no other configurations are used by OutputFormat
+ */
+ protected static Instance getInstance(Configuration conf) {
return new ZooKeeperInstance(conf.get(INSTANCE_NAME), conf.get(ZOOKEEPERS));
}
}