1
0

CodeStyle formatting to conform to basic Checkstyle rules.

The code-style rules follow google style with some changes:

1. Increase line length from 100 to 120
2. Disable JavaDoc related checkstyles as this needs more manual work.

Both source and test code are checked for code-style
This commit is contained in:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -23,8 +23,7 @@ import org.apache.log4j.Logger;
public class HoodieHiveUtil {
public static final Logger LOG =
LogManager.getLogger(HoodieHiveUtil.class);
public static final Logger LOG = LogManager.getLogger(HoodieHiveUtil.class);
public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode";
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";

View File

@@ -71,8 +71,7 @@ import parquet.io.api.Binary;
* Hoodie/Non-Hoodie datasets
*/
@UseFileSplitsFromInputFormat
public class HoodieInputFormat extends MapredParquetInputFormat
implements Configurable {
public class HoodieInputFormat extends MapredParquetInputFormat implements Configurable {
public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class);
@@ -99,7 +98,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat
}
String tableName = metadata.getTableConfig().getTableName();
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
// Get all commits, delta commits, compactions, as all of them produce a base parquet file today
// Get all commits, delta commits, compactions, as all of them produce a base parquet file
// today
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants();
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metadata,
@@ -112,19 +112,16 @@ public class HoodieInputFormat extends MapredParquetInputFormat
// Total number of commits to return in this batch. Set this to -1 to get all the commits.
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
List<String> commitsToReturn =
timeline.findInstantsAfter(lastIncrementalTs, maxCommits).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles = roView
.getLatestDataFilesInRange(commitsToReturn)
List<String> commitsToReturn = timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<HoodieDataFile> filteredFiles = roView.getLatestDataFilesInRange(commitsToReturn)
.collect(Collectors.toList());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
filteredFile = checkFileStatus(filteredFile);
returns.add(filteredFile.getFileStatus());
}
LOG.info(
"Total paths to process after hoodie incremental filter " + filteredFiles.size());
LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size());
} else {
// filter files on the latest commit found
List<HoodieDataFile> filteredFiles = roView.getLatestDataFiles()
@@ -171,13 +168,13 @@ public class HoodieInputFormat extends MapredParquetInputFormat
String nonHoodieBasePath = null;
for (FileStatus status : fileStatuses) {
if (!status.getPath().getName().endsWith(".parquet")) {
//FIXME(vc): skip non parquet files for now. This wont be needed once log file name start with "."
//FIXME(vc): skip non parquet files for now. This wont be needed once log file name start
// with "."
continue;
}
if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath()
.toString()
.contains(nonHoodieBasePath)) || (metadata != null && !status.getPath().toString()
.contains(metadata.getBasePath()))) {
.toString().contains(nonHoodieBasePath)) || (metadata != null && !status.getPath()
.toString().contains(metadata.getBasePath()))) {
try {
metadata = getTableMetaClient(status.getPath().getFileSystem(conf),
status.getPath().getParent());
@@ -185,8 +182,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
} catch (InvalidDatasetException e) {
LOG.info("Handling a non-hoodie path " + status.getPath());
metadata = null;
nonHoodieBasePath =
status.getPath().getParent().toString();
nonHoodieBasePath = status.getPath().getParent().toString();
}
if (!grouped.containsKey(metadata)) {
grouped.put(metadata, new ArrayList<>());
@@ -209,17 +205,17 @@ public class HoodieInputFormat extends MapredParquetInputFormat
public RecordReader<Void, ArrayWritable> getRecordReader(final InputSplit split,
final JobConf job, final Reporter reporter) throws IOException {
// TODO enable automatic predicate pushdown after fixing issues
// FileSplit fileSplit = (FileSplit) split;
// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent());
// String tableName = metadata.getTableName();
// String mode = HoodieHiveUtil.readMode(job, tableName);
// FileSplit fileSplit = (FileSplit) split;
// HoodieTableMetadata metadata = getTableMetadata(fileSplit.getPath().getParent());
// String tableName = metadata.getTableName();
// String mode = HoodieHiveUtil.readMode(job, tableName);
// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
// LOG.info("Setting parquet predicate push down as " + predicate);
// ParquetInputFormat.setFilterPredicate(job, predicate);
// if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// FilterPredicate predicate = constructHoodiePredicate(job, tableName, split);
// LOG.info("Setting parquet predicate push down as " + predicate);
// ParquetInputFormat.setFilterPredicate(job, predicate);
//clearOutExistingPredicate(job);
// }
// }
return super.getRecordReader(split, job, reporter);
}
@@ -236,9 +232,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat
* `hoodie_commit_time` > 'start_commit_time' and ANDs with the existing predicate if one is
* present already.
*/
private FilterPredicate constructHoodiePredicate(JobConf job,
String tableName,
InputSplit split) throws IOException {
private FilterPredicate constructHoodiePredicate(JobConf job, String tableName, InputSplit split)
throws IOException {
FilterPredicate commitTimePushdown = constructCommitTimePushdownPredicate(job, tableName);
LOG.info("Commit time predicate - " + commitTimePushdown.toString());
FilterPredicate existingPushdown = constructHQLPushdownPredicate(job, split);
@@ -262,21 +257,19 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|| columnNamesString.isEmpty()) {
return null;
} else {
SearchArgument sarg =
SearchArgumentFactory.create(Utilities.deserializeExpression(serializedPushdown));
SearchArgument sarg = SearchArgumentFactory
.create(Utilities.deserializeExpression(serializedPushdown));
final Path finalPath = ((FileSplit) split).getPath();
final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(job, finalPath);
final FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
return ParquetFilterPredicateConverter
.toFilterPredicate(sarg, fileMetaData.getSchema());
return ParquetFilterPredicateConverter.toFilterPredicate(sarg, fileMetaData.getSchema());
}
}
private FilterPredicate constructCommitTimePushdownPredicate(JobConf job, String tableName)
throws IOException {
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
Operators.BinaryColumn sequenceColumn =
binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
Operators.BinaryColumn sequenceColumn = binaryColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
FilterPredicate p = gt(sequenceColumn, Binary.fromString(lastIncrementalTs));
LOG.info("Setting predicate in InputFormat " + p.toString());
return p;

View File

@@ -13,6 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.hadoop;
import com.uber.hoodie.common.model.HoodieDataFile;
@@ -37,12 +38,12 @@ import org.apache.hadoop.fs.PathFilter;
/**
* Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path -
* Non-Hoodie dataset = then always accept
*
* <p>
* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then
* you should be able to query both hoodie and non-hoodie datasets as you would normally do.
*
* hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop.HoodieROTablePathFilter.class,
* org.apache.hadoop.fs.PathFilter.class)
* <p>
* hadoopConf.setClass("mapreduce.input.pathFilter.class", com.uber.hoodie.hadoop
* .HoodieROTablePathFilter.class, org.apache.hadoop.fs.PathFilter.class)
*/
public class HoodieROTablePathFilter implements PathFilter, Serializable {
@@ -104,8 +105,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
if (hoodiePathCache.containsKey(folder.toString())) {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n",
path,
LOG.debug(String.format("%s Hoodie path checked against cache, accept => %s \n", path,
hoodiePathCache.get(folder.toString()).contains(path)));
}
return hoodiePathCache.get(folder.toString()).contains(path);
@@ -123,37 +123,33 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
if (baseDir != null) {
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(),
baseDir.toString());
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline()
.filterCompletedInstants(),
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
fs.listStatus(folder));
List<HoodieDataFile> latestFiles = fsView
.getLatestDataFiles()
List<HoodieDataFile> latestFiles = fsView.getLatestDataFiles()
.collect(Collectors.toList());
// populate the cache
if (!hoodiePathCache.containsKey(folder.toString())) {
hoodiePathCache.put(folder.toString(), new HashSet<>());
}
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() +
", caching " + latestFiles.size() + " files under " + folder);
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching "
+ latestFiles.size() + " files under " + folder);
for (HoodieDataFile lfile : latestFiles) {
hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath()));
}
// accept the path, if its among the latest files.
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("%s checked after cache population, accept => %s \n",
path,
LOG.debug(String.format("%s checked after cache population, accept => %s \n", path,
hoodiePathCache.get(folder.toString()).contains(path)));
}
return hoodiePathCache.get(folder.toString()).contains(path);
} catch (DatasetNotFoundException e) {
// Non-hoodie path, accept it.
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("(1) Caching non-hoodie path under %s \n",
folder.toString()));
LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", folder.toString()));
}
nonHoodiePathCache.add(folder.toString());
return true;

View File

@@ -79,8 +79,7 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
// TODO(vc): Should we handle also non-hoodie splits here?
Map<String, HoodieTableMetaClient> metaClientMap = new HashMap<>();
Map<Path, HoodieTableMetaClient> partitionsToMetaClient = partitionsToParquetSplits.keySet()
.stream()
.collect(Collectors.toMap(Function.identity(), p -> {
.stream().collect(Collectors.toMap(Function.identity(), p -> {
// find if we have a metaclient already for this partition.
Optional<String> matchingBasePath = metaClientMap.keySet().stream()
.filter(basePath -> p.toString().startsWith(basePath)).findFirst();
@@ -97,7 +96,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
}
}));
// for all unique split parents, obtain all delta files based on delta commit timeline, grouped on file id
// for all unique split parents, obtain all delta files based on delta commit timeline,
// grouped on file id
List<HoodieRealtimeFileSplit> rtSplits = new ArrayList<>();
partitionsToParquetSplits.keySet().stream().forEach(partitionPath -> {
// for each partition path obtain the data & log file groupings, then map back to inputsplits
@@ -119,14 +119,13 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
dataFileSplits.forEach(split -> {
try {
List<String> logFilePaths = fileSlice.getLogFiles()
.map(logFile -> logFile.getPath().toString())
.collect(Collectors.toList());
// Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table
String maxCommitTime = metaClient.getActiveTimeline()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
// Get the maxCommit from the last delta or compaction or commit - when
// bootstrapped from COW table
String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION)).filterCompletedInstants().lastInstant()
.get().getTimestamp();
rtSplits.add(
new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths,
maxCommitTime));
@@ -147,7 +146,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
@Override
public FileStatus[] listStatus(JobConf job) throws IOException {
// Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit timeline.
// Call the HoodieInputFormat::listStatus to obtain all latest parquet files, based on commit
// timeline.
return super.listStatus(job);
}
@@ -170,12 +170,11 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
if (!readColNames.contains(fieldName)) {
// If not already in the list - then add it
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR,
readColNamesPrefix + fieldName);
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex);
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("Adding extra column " + fieldName
+ ", to enable log merging cols (%s) ids (%s) ",
LOG.debug(String.format(
"Adding extra column " + fieldName + ", to enable log merging cols (%s) ids (%s) ",
conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
conf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)));
}
@@ -189,15 +188,14 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf
HOODIE_RECORD_KEY_COL_POS);
configuration = addProjectionField(configuration, HoodieRecord.COMMIT_TIME_METADATA_FIELD,
HOODIE_COMMIT_TIME_COL_POS);
configuration = addProjectionField(configuration,
HoodieRecord.PARTITION_PATH_METADATA_FIELD, HOODIE_PARTITION_PATH_COL_POS);
configuration = addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD,
HOODIE_PARTITION_PATH_COL_POS);
return configuration;
}
@Override
public RecordReader<Void, ArrayWritable> getRecordReader(final InputSplit split,
final JobConf job,
final Reporter reporter) throws IOException {
final JobConf job, final Reporter reporter) throws IOException {
LOG.info("Creating record reader with readCols :" + job
.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
// sanity check

View File

@@ -73,8 +73,10 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
public static final String DEFAULT_COMPACTION_MEMORY_FRACTION = "0.75";
// used to choose a trade off between IO vs Memory when performing compaction process
// Depending on outputfile size and memory provided, choose true to avoid OOM for large file size + small memory
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "compaction.lazy.block.read.enabled";
// Depending on outputfile size and memory provided, choose true to avoid OOM for large file
// size + small memory
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
"compaction.lazy.block" + ".read.enabled";
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "true";
public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class);
@@ -82,8 +84,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
private final HashMap<String, ArrayWritable> deltaRecordMap;
private final MessageType baseFileSchema;
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split,
JobConf job,
public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job,
RecordReader<Void, ArrayWritable> realReader) {
this.split = split;
this.jobConf = job;
@@ -106,11 +107,9 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
*/
private static MessageType readSchema(Configuration conf, Path parquetFilePath) {
try {
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData()
.getSchema();
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema();
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
e);
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e);
}
}
@@ -125,27 +124,27 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR),
jobConf.get("partition_columns", ""));
// TODO(vc): In the future, the reader schema should be updated based on log files & be able to null out fields not present before
// TODO(vc): In the future, the reader schema should be updated based on log files & be able
// to null out fields not present before
Schema readerSchema = generateProjectionSchema(writerSchema, projectionFields);
LOG.info(
String.format("About to read compacted logs %s for base split %s, projecting cols %s",
split.getDeltaFilePaths(), split.getPath(), projectionFields));
HoodieCompactedLogRecordScanner compactedLogRecordScanner =
new HoodieCompactedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
split.getBasePath(),
split.getDeltaFilePaths(),
readerSchema, split.getMaxCommitTime(),
(long) Math.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
*jobConf.getMemoryForMapTask()),
Boolean.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), false);
LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s",
split.getDeltaFilePaths(), split.getPath(), projectionFields));
HoodieCompactedLogRecordScanner compactedLogRecordScanner = new HoodieCompactedLogRecordScanner(
FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(),
split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime(), (long) Math.ceil(Double
.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
* jobConf.getMemoryForMapTask()), Boolean.valueOf(jobConf
.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
false);
// NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit
// but can return records for completed commits > the commit we are trying to read (if using readCommit() API)
// but can return records for completed commits > the commit we are trying to read (if using
// readCommit() API)
for (HoodieRecord<? extends HoodieRecordPayload> hoodieRecord : compactedLogRecordScanner) {
GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema)
.get();
GenericRecord rec = (GenericRecord) hoodieRecord.getData().getInsertValue(readerSchema).get();
String key = hoodieRecord.getRecordKey();
// we assume, a later safe record in the log, is newer than what we have in the map & replace it.
// we assume, a later safe record in the log, is newer than what we have in the map &
// replace it.
// TODO : handle deletes here
ArrayWritable aWritable = (ArrayWritable) avroToArrayWritable(rec, writerSchema);
deltaRecordMap.put(key, aWritable);
@@ -180,14 +179,13 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
Set<String> partitioningFields = Arrays.stream(partitioningFieldsCsv.split(","))
.collect(Collectors.toSet());
List<String> fieldNames = Arrays.stream(fieldNameCsv.split(","))
.filter(fn -> !partitioningFields.contains(fn)).collect(
Collectors.toList());
.filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList());
// Hive does not provide ids for partitioning fields, so check for lengths excluding that.
if (fieldNames.size() != fieldOrders.length) {
throw new HoodieException(String.format(
"Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d",
fieldNames.size(), fieldOrders.length));
throw new HoodieException(String
.format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d",
fieldNames.size(), fieldOrders.length));
}
TreeMap<Integer, String> orderedFieldMap = new TreeMap<>();
for (int ox = 0; ox < fieldOrders.length; ox++) {
@@ -287,26 +285,28 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
}
case FIXED:
return new BytesWritable(((GenericFixed) value).bytes());
default:
return null;
}
return null;
}
@Override
public boolean next(Void aVoid, ArrayWritable arrayWritable) throws IOException {
// Call the underlying parquetReader.next - which may replace the passed in ArrayWritable with a new block of values
// Call the underlying parquetReader.next - which may replace the passed in ArrayWritable
// with a new block of values
boolean result = this.parquetReader.next(aVoid, arrayWritable);
if (!result) {
// if the result is false, then there are no more records
return false;
} else {
// TODO(VC): Right now, we assume all records in log, have a matching base record. (which would be true until we have a way to index logs too)
// TODO(VC): Right now, we assume all records in log, have a matching base record. (which
// would be true until we have a way to index logs too)
// return from delta records map if we have some match.
String key = arrayWritable.get()[HoodieRealtimeInputFormat.HOODIE_RECORD_KEY_COL_POS]
.toString();
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("key %s, base values: %s, log values: %s",
key, arrayWritableToString(arrayWritable),
arrayWritableToString(deltaRecordMap.get(key))));
LOG.debug(String.format("key %s, base values: %s, log values: %s", key,
arrayWritableToString(arrayWritable), arrayWritableToString(deltaRecordMap.get(key))));
}
if (deltaRecordMap.containsKey(key)) {
// TODO(NA): Invoke preCombine here by converting arrayWritable to Avro ?