blob: 94a53d493a706be58b5c7d1f5ca416d28e156db2 [file] [log] [blame]
/**
* 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.distributedlog.mapreduce;
import com.google.common.collect.Lists;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.DistributedLogManager;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.LogRecordWithDLSN;
import org.apache.distributedlog.LogSegmentMetadata;
import org.apache.distributedlog.impl.BKNamespaceDriver;
import org.apache.distributedlog.namespace.DistributedLogNamespace;
import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeperAccessor;
import org.apache.bookkeeper.client.LedgerMetadata;
import org.apache.bookkeeper.meta.LedgerManager;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import java.io.IOException;
import java.net.URI;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
/**
* InputFormat to read data from a distributedlog stream.
*/
public class DistributedLogInputFormat
extends InputFormat<DLSN, LogRecordWithDLSN> implements Configurable {
private static final String DL_URI = "distributedlog.uri";
private static final String DL_STREAM = "distributedlog.stream";
protected Configuration conf;
protected DistributedLogConfiguration dlConf;
protected URI dlUri;
protected DistributedLogNamespace namespace;
protected String streamName;
protected DistributedLogManager dlm;
/** {@inheritDoc} */
@Override
public void setConf(Configuration configuration) {
this.conf = configuration;
dlConf = new DistributedLogConfiguration();
dlUri = URI.create(configuration.get(DL_URI, ""));
streamName = configuration.get(DL_STREAM, "");
try {
namespace = DistributedLogNamespaceBuilder.newBuilder()
.conf(dlConf)
.uri(dlUri)
.build();
dlm = namespace.openLog(streamName);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public Configuration getConf() {
return conf;
}
@Override
public List<InputSplit> getSplits(JobContext jobContext)
throws IOException, InterruptedException {
List<LogSegmentMetadata> segments = dlm.getLogSegments();
List<InputSplit> inputSplits = Lists.newArrayListWithCapacity(segments.size());
BookKeeper bk = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get();
LedgerManager lm = BookKeeperAccessor.getLedgerManager(bk);
final AtomicInteger rcHolder = new AtomicInteger(0);
final AtomicReference<LedgerMetadata> metadataHolder = new AtomicReference<LedgerMetadata>(null);
for (LogSegmentMetadata segment : segments) {
final CountDownLatch latch = new CountDownLatch(1);
lm.readLedgerMetadata(segment.getLogSegmentId(),
new BookkeeperInternalCallbacks.GenericCallback<LedgerMetadata>() {
@Override
public void operationComplete(int rc, LedgerMetadata ledgerMetadata) {
metadataHolder.set(ledgerMetadata);
rcHolder.set(rc);
latch.countDown();
}
});
latch.await();
if (BKException.Code.OK != rcHolder.get()) {
throw new IOException("Faild to get log segment metadata for " + segment + " : "
+ BKException.getMessage(rcHolder.get()));
}
inputSplits.add(new LogSegmentSplit(segment, metadataHolder.get()));
}
return inputSplits;
}
@Override
public RecordReader<DLSN, LogRecordWithDLSN> createRecordReader(
InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
throws IOException, InterruptedException {
return new LogSegmentReader(
streamName,
dlConf,
((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get(),
(LogSegmentSplit) inputSplit);
}
}