HDFS-10957. Retire BKJM from trunk (Vinayakumar B)
This commit is contained in:
parent
35b9d7de9f
commit
311954883f
|
@ -1,66 +0,0 @@
|
|||
This module provides a BookKeeper backend for HFDS Namenode write
|
||||
ahead logging.
|
||||
|
||||
BookKeeper is a highly available distributed write ahead logging
|
||||
system. For more details, see
|
||||
|
||||
http://zookeeper.apache.org/bookkeeper
|
||||
|
||||
-------------------------------------------------------------------------------
|
||||
How do I build?
|
||||
|
||||
To generate the distribution packages for BK journal, do the
|
||||
following.
|
||||
|
||||
$ mvn clean package -Pdist
|
||||
|
||||
This will generate a jar with all the dependencies needed by the journal
|
||||
manager,
|
||||
|
||||
target/hadoop-hdfs-bkjournal-<VERSION>.jar
|
||||
|
||||
Note that the -Pdist part of the build command is important, as otherwise
|
||||
the dependencies would not be packaged in the jar.
|
||||
|
||||
-------------------------------------------------------------------------------
|
||||
How do I use the BookKeeper Journal?
|
||||
|
||||
To run a HDFS namenode using BookKeeper as a backend, copy the bkjournal
|
||||
jar, generated above, into the lib directory of hdfs. In the standard
|
||||
distribution of HDFS, this is at $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
|
||||
|
||||
cp target/hadoop-hdfs-bkjournal-<VERSION>.jar \
|
||||
$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
|
||||
|
||||
Then, in hdfs-site.xml, set the following properties.
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.edits.dir</name>
|
||||
<value>bookkeeper://localhost:2181/bkjournal,file:///path/for/edits</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.edits.journal-plugin.bookkeeper</name>
|
||||
<value>org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager</value>
|
||||
</property>
|
||||
|
||||
In this example, the namenode is configured to use 2 write ahead
|
||||
logging devices. One writes to BookKeeper and the other to a local
|
||||
file system. At the moment is is not possible to only write to
|
||||
BookKeeper, as the resource checker explicitly checked for local
|
||||
disks currently.
|
||||
|
||||
The given example, configures the namenode to look for the journal
|
||||
metadata at the path /bkjournal on the a standalone zookeeper ensemble
|
||||
at localhost:2181. To configure a multiple host zookeeper ensemble,
|
||||
separate the hosts with semicolons. For example, if you have 3
|
||||
zookeeper servers, zk1, zk2 & zk3, each listening on port 2181, you
|
||||
would specify this with
|
||||
|
||||
bookkeeper://zk1:2181;zk2:2181;zk3:2181/bkjournal
|
||||
|
||||
The final part /bkjournal specifies the znode in zookeeper where
|
||||
ledger metadata will be store. Administrators can set this to anything
|
||||
they wish.
|
||||
|
||||
|
|
@ -1,5 +0,0 @@
|
|||
<FindBugsFilter>
|
||||
<Match>
|
||||
<Class name="~org.apache.hadoop.contrib.bkjournal.BKJournalProtos.*" />
|
||||
</Match>
|
||||
</FindBugsFilter>
|
|
@ -1,175 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
Licensed 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. See accompanying LICENSE file.
|
||||
-->
|
||||
<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/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-project</artifactId>
|
||||
<version>3.0.0-alpha2-SNAPSHOT</version>
|
||||
<relativePath>../../../../../hadoop-project</relativePath>
|
||||
</parent>
|
||||
|
||||
<groupId>org.apache.hadoop.contrib</groupId>
|
||||
<artifactId>hadoop-hdfs-bkjournal</artifactId>
|
||||
<version>3.0.0-alpha2-SNAPSHOT</version>
|
||||
<description>Apache Hadoop HDFS BookKeeper Journal</description>
|
||||
<name>Apache Hadoop HDFS BookKeeper Journal</name>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<hadoop.component>hdfs</hadoop.component>
|
||||
<hadoop.common.build.dir>${basedir}/../../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>commons-logging</groupId>
|
||||
<artifactId>commons-logging</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.bookkeeper</groupId>
|
||||
<artifactId>bookkeeper-server</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.zookeeper</groupId>
|
||||
<artifactId>zookeeper</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-maven-plugins</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>compile-protoc</id>
|
||||
<phase>generate-sources</phase>
|
||||
<goals>
|
||||
<goal>protoc</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<protocVersion>${protobuf.version}</protocVersion>
|
||||
<protocCommand>${protoc.path}</protocCommand>
|
||||
<imports>
|
||||
<param>${basedir}/../../../../../hadoop-common-project/hadoop-common/src/main/proto</param>
|
||||
<param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto</param>
|
||||
<param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs/src/main/proto</param>
|
||||
<param>${basedir}/src/main/proto</param>
|
||||
</imports>
|
||||
<source>
|
||||
<directory>${basedir}/src/main/proto</directory>
|
||||
<includes>
|
||||
<include>bkjournal.proto</include>
|
||||
</includes>
|
||||
</source>
|
||||
<output>${project.build.directory}/generated-sources/java</output>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>findbugs-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<exclude>dev-support/findbugsExcludeFile.xml</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
<profiles>
|
||||
<profile>
|
||||
<id>dist</id>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<version>2.8</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>dist</id>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>copy</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<artifactItems>
|
||||
<artifactItem>
|
||||
<groupId>org.apache.bookkeeper</groupId>
|
||||
<artifactId>bookkeeper-server</artifactId>
|
||||
<type>jar</type>
|
||||
</artifactItem>
|
||||
</artifactItems>
|
||||
<outputDirectory>${project.build.directory}/lib</outputDirectory>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</profile>
|
||||
</profiles>
|
||||
</project>
|
|
@ -1,264 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Enumeration;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader;
|
||||
import org.apache.bookkeeper.client.LedgerHandle;
|
||||
import org.apache.bookkeeper.client.LedgerEntry;
|
||||
import org.apache.bookkeeper.client.BKException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
/**
|
||||
* Input stream which reads from a BookKeeper ledger.
|
||||
*/
|
||||
class BookKeeperEditLogInputStream extends EditLogInputStream {
|
||||
static final Log LOG = LogFactory.getLog(BookKeeperEditLogInputStream.class);
|
||||
|
||||
private final long firstTxId;
|
||||
private final long lastTxId;
|
||||
private final int logVersion;
|
||||
private final boolean inProgress;
|
||||
private final LedgerHandle lh;
|
||||
|
||||
private final FSEditLogOp.Reader reader;
|
||||
private final FSEditLogLoader.PositionTrackingInputStream tracker;
|
||||
|
||||
/**
|
||||
* Construct BookKeeper edit log input stream.
|
||||
* Starts reading from the first entry of the ledger.
|
||||
*/
|
||||
BookKeeperEditLogInputStream(final LedgerHandle lh,
|
||||
final EditLogLedgerMetadata metadata)
|
||||
throws IOException {
|
||||
this(lh, metadata, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct BookKeeper edit log input stream.
|
||||
* Starts reading from firstBookKeeperEntry. This allows the stream
|
||||
* to take a shortcut during recovery, as it doesn't have to read
|
||||
* every edit log transaction to find out what the last one is.
|
||||
*/
|
||||
BookKeeperEditLogInputStream(LedgerHandle lh, EditLogLedgerMetadata metadata,
|
||||
long firstBookKeeperEntry)
|
||||
throws IOException {
|
||||
this.lh = lh;
|
||||
this.firstTxId = metadata.getFirstTxId();
|
||||
this.lastTxId = metadata.getLastTxId();
|
||||
this.logVersion = metadata.getDataLayoutVersion();
|
||||
this.inProgress = metadata.isInProgress();
|
||||
|
||||
if (firstBookKeeperEntry < 0
|
||||
|| firstBookKeeperEntry > lh.getLastAddConfirmed()) {
|
||||
throw new IOException("Invalid first bk entry to read: "
|
||||
+ firstBookKeeperEntry + ", LAC: " + lh.getLastAddConfirmed());
|
||||
}
|
||||
BufferedInputStream bin = new BufferedInputStream(
|
||||
new LedgerInputStream(lh, firstBookKeeperEntry));
|
||||
tracker = new FSEditLogLoader.PositionTrackingInputStream(bin);
|
||||
DataInputStream in = new DataInputStream(tracker);
|
||||
|
||||
reader = FSEditLogOp.Reader.create(in, tracker, logVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFirstTxId() {
|
||||
return firstTxId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLastTxId() {
|
||||
return lastTxId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getVersion(boolean verifyVersion) throws IOException {
|
||||
return logVersion;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FSEditLogOp nextOp() throws IOException {
|
||||
return reader.readOp(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
lh.close();
|
||||
} catch (BKException e) {
|
||||
throw new IOException("Exception closing ledger", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted closing ledger", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPosition() {
|
||||
return tracker.getPos();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long length() throws IOException {
|
||||
return lh.getLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return String.format(
|
||||
"BookKeeperLedger[ledgerId=%d,firstTxId=%d,lastTxId=%d]", lh.getId(),
|
||||
firstTxId, lastTxId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInProgress() {
|
||||
return inProgress;
|
||||
}
|
||||
|
||||
/**
|
||||
* Skip forward to specified transaction id.
|
||||
* Currently we do this by just iterating forward.
|
||||
* If this proves to be too expensive, this can be reimplemented
|
||||
* with a binary search over bk entries
|
||||
*/
|
||||
public void skipTo(long txId) throws IOException {
|
||||
long numToSkip = getFirstTxId() - txId;
|
||||
|
||||
FSEditLogOp op = null;
|
||||
for (long i = 0; i < numToSkip; i++) {
|
||||
op = readOp();
|
||||
}
|
||||
if (op != null && op.getTransactionId() != txId-1) {
|
||||
throw new IOException("Corrupt stream, expected txid "
|
||||
+ (txId-1) + ", got " + op.getTransactionId());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return ("BookKeeperEditLogInputStream {" + this.getName() + "}");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMaxOpSize(int maxOpSize) {
|
||||
reader.setMaxOpSize(maxOpSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocalLog() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Input stream implementation which can be used by
|
||||
* FSEditLogOp.Reader
|
||||
*/
|
||||
private static class LedgerInputStream extends InputStream {
|
||||
private long readEntries;
|
||||
private InputStream entryStream = null;
|
||||
private final LedgerHandle lh;
|
||||
private final long maxEntry;
|
||||
|
||||
/**
|
||||
* Construct ledger input stream
|
||||
* @param lh the ledger handle to read from
|
||||
* @param firstBookKeeperEntry ledger entry to start reading from
|
||||
*/
|
||||
LedgerInputStream(LedgerHandle lh, long firstBookKeeperEntry)
|
||||
throws IOException {
|
||||
this.lh = lh;
|
||||
readEntries = firstBookKeeperEntry;
|
||||
|
||||
maxEntry = lh.getLastAddConfirmed();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get input stream representing next entry in the
|
||||
* ledger.
|
||||
* @return input stream, or null if no more entries
|
||||
*/
|
||||
private InputStream nextStream() throws IOException {
|
||||
try {
|
||||
if (readEntries > maxEntry) {
|
||||
return null;
|
||||
}
|
||||
Enumeration<LedgerEntry> entries
|
||||
= lh.readEntries(readEntries, readEntries);
|
||||
readEntries++;
|
||||
if (entries.hasMoreElements()) {
|
||||
LedgerEntry e = entries.nextElement();
|
||||
assert !entries.hasMoreElements();
|
||||
return e.getEntryInputStream();
|
||||
}
|
||||
} catch (BKException e) {
|
||||
throw new IOException("Error reading entries from bookkeeper", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted reading entries from bookkeeper", e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
byte[] b = new byte[1];
|
||||
if (read(b, 0, 1) != 1) {
|
||||
return -1;
|
||||
} else {
|
||||
return b[0];
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
try {
|
||||
int read = 0;
|
||||
if (entryStream == null) {
|
||||
entryStream = nextStream();
|
||||
if (entryStream == null) {
|
||||
return read;
|
||||
}
|
||||
}
|
||||
|
||||
while (read < len) {
|
||||
int thisread = entryStream.read(b, off+read, (len-read));
|
||||
if (thisread == -1) {
|
||||
entryStream = nextStream();
|
||||
if (entryStream == null) {
|
||||
return read;
|
||||
}
|
||||
} else {
|
||||
read += thisread;
|
||||
}
|
||||
}
|
||||
return read;
|
||||
} catch (IOException e) {
|
||||
throw e;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,188 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.bookkeeper.client.LedgerHandle;
|
||||
import org.apache.bookkeeper.client.BKException;
|
||||
import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
/**
|
||||
* Output stream for BookKeeper Journal.
|
||||
* Multiple complete edit log entries are packed into a single bookkeeper
|
||||
* entry before sending it over the network. The fact that the edit log entries
|
||||
* are complete in the bookkeeper entries means that each bookkeeper log entry
|
||||
*can be read as a complete edit log. This is useful for recover, as we don't
|
||||
* need to read through the entire edit log segment to get the last written
|
||||
* entry.
|
||||
*/
|
||||
class BookKeeperEditLogOutputStream
|
||||
extends EditLogOutputStream implements AddCallback {
|
||||
static final Log LOG = LogFactory.getLog(BookKeeperEditLogOutputStream.class);
|
||||
|
||||
private final DataOutputBuffer bufCurrent;
|
||||
private final AtomicInteger outstandingRequests;
|
||||
private final int transmissionThreshold;
|
||||
private final LedgerHandle lh;
|
||||
private CountDownLatch syncLatch;
|
||||
private final AtomicInteger transmitResult
|
||||
= new AtomicInteger(BKException.Code.OK);
|
||||
private final Writer writer;
|
||||
|
||||
/**
|
||||
* Construct an edit log output stream which writes to a ledger.
|
||||
|
||||
*/
|
||||
protected BookKeeperEditLogOutputStream(Configuration conf, LedgerHandle lh)
|
||||
throws IOException {
|
||||
super();
|
||||
|
||||
bufCurrent = new DataOutputBuffer();
|
||||
outstandingRequests = new AtomicInteger(0);
|
||||
syncLatch = null;
|
||||
this.lh = lh;
|
||||
this.writer = new Writer(bufCurrent);
|
||||
this.transmissionThreshold
|
||||
= conf.getInt(BookKeeperJournalManager.BKJM_OUTPUT_BUFFER_SIZE,
|
||||
BookKeeperJournalManager.BKJM_OUTPUT_BUFFER_SIZE_DEFAULT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void create(int layoutVersion) throws IOException {
|
||||
// noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
setReadyToFlush();
|
||||
flushAndSync(true);
|
||||
try {
|
||||
lh.close();
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted waiting on close", ie);
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("BookKeeper error during close", bke);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() throws IOException {
|
||||
try {
|
||||
lh.close();
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted waiting on close", ie);
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("BookKeeper error during abort", bke);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeRaw(final byte[] data, int off, int len) throws IOException {
|
||||
throw new IOException("Not supported for BK");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(FSEditLogOp op) throws IOException {
|
||||
writer.writeOp(op);
|
||||
|
||||
if (bufCurrent.getLength() > transmissionThreshold) {
|
||||
transmit();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setReadyToFlush() throws IOException {
|
||||
transmit();
|
||||
|
||||
synchronized (this) {
|
||||
syncLatch = new CountDownLatch(outstandingRequests.get());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flushAndSync(boolean durable) throws IOException {
|
||||
assert(syncLatch != null);
|
||||
try {
|
||||
syncLatch.await();
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted waiting on latch", ie);
|
||||
}
|
||||
if (transmitResult.get() != BKException.Code.OK) {
|
||||
throw new IOException("Failed to write to bookkeeper; Error is ("
|
||||
+ transmitResult.get() + ") "
|
||||
+ BKException.getMessage(transmitResult.get()));
|
||||
}
|
||||
|
||||
syncLatch = null;
|
||||
// wait for whatever we wait on
|
||||
}
|
||||
|
||||
/**
|
||||
* Transmit the current buffer to bookkeeper.
|
||||
* Synchronised at the FSEditLog level. #write() and #setReadyToFlush()
|
||||
* are never called at the same time.
|
||||
*/
|
||||
private void transmit() throws IOException {
|
||||
if (!transmitResult.compareAndSet(BKException.Code.OK,
|
||||
BKException.Code.OK)) {
|
||||
throw new IOException("Trying to write to an errored stream;"
|
||||
+ " Error code : (" + transmitResult.get()
|
||||
+ ") " + BKException.getMessage(transmitResult.get()));
|
||||
}
|
||||
if (bufCurrent.getLength() > 0) {
|
||||
byte[] entry = Arrays.copyOf(bufCurrent.getData(),
|
||||
bufCurrent.getLength());
|
||||
lh.asyncAddEntry(entry, this, null);
|
||||
bufCurrent.reset();
|
||||
outstandingRequests.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addComplete(int rc, LedgerHandle handle,
|
||||
long entryId, Object ctx) {
|
||||
synchronized(this) {
|
||||
outstandingRequests.decrementAndGet();
|
||||
if (!transmitResult.compareAndSet(BKException.Code.OK, rc)) {
|
||||
LOG.warn("Tried to set transmit result to (" + rc + ") \""
|
||||
+ BKException.getMessage(rc) + "\""
|
||||
+ " but is already (" + transmitResult.get() + ") \""
|
||||
+ BKException.getMessage(transmitResult.get()) + "\"");
|
||||
}
|
||||
CountDownLatch l = syncLatch;
|
||||
if (l != null) {
|
||||
l.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,893 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage;
|
||||
import org.apache.hadoop.hdfs.server.common.StorageInfo;
|
||||
import org.apache.hadoop.hdfs.server.namenode.JournalManager;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.bookkeeper.conf.ClientConfiguration;
|
||||
import org.apache.bookkeeper.client.BKException;
|
||||
import org.apache.bookkeeper.client.BookKeeper;
|
||||
import org.apache.bookkeeper.client.LedgerHandle;
|
||||
import org.apache.bookkeeper.util.ZkUtils;
|
||||
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.AsyncCallback.StringCallback;
|
||||
import org.apache.zookeeper.ZKUtil;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.io.IOException;
|
||||
|
||||
import java.net.URI;
|
||||
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.VersionProto;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import static com.google.common.base.Charsets.UTF_8;
|
||||
|
||||
import org.apache.commons.io.Charsets;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
/**
|
||||
* BookKeeper Journal Manager
|
||||
*
|
||||
* To use, add the following to hdfs-site.xml.
|
||||
* <pre>
|
||||
* {@code
|
||||
* <property>
|
||||
* <name>dfs.namenode.edits.dir</name>
|
||||
* <value>bookkeeper://zk1:2181;zk2:2181;zk3:2181/hdfsjournal</value>
|
||||
* </property>
|
||||
*
|
||||
* <property>
|
||||
* <name>dfs.namenode.edits.journal-plugin.bookkeeper</name>
|
||||
* <value>org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager</value>
|
||||
* </property>
|
||||
* }
|
||||
* </pre>
|
||||
* The URI format for bookkeeper is bookkeeper://[zkEnsemble]/[rootZnode]
|
||||
* [zookkeeper ensemble] is a list of semi-colon separated, zookeeper host:port
|
||||
* pairs. In the example above there are 3 servers, in the ensemble,
|
||||
* zk1, zk2 & zk3, each one listening on port 2181.
|
||||
*
|
||||
* [root znode] is the path of the zookeeper znode, under which the editlog
|
||||
* information will be stored.
|
||||
*
|
||||
* Other configuration options are:
|
||||
* <ul>
|
||||
* <li><b>dfs.namenode.bookkeeperjournal.output-buffer-size</b>
|
||||
* Number of bytes a bookkeeper journal stream will buffer before
|
||||
* forcing a flush. Default is 1024.</li>
|
||||
* <li><b>dfs.namenode.bookkeeperjournal.ensemble-size</b>
|
||||
* Number of bookkeeper servers in edit log ledger ensembles. This
|
||||
* is the number of bookkeeper servers which need to be available
|
||||
* for the ledger to be writable. Default is 3.</li>
|
||||
* <li><b>dfs.namenode.bookkeeperjournal.quorum-size</b>
|
||||
* Number of bookkeeper servers in the write quorum. This is the
|
||||
* number of bookkeeper servers which must have acknowledged the
|
||||
* write of an entry before it is considered written.
|
||||
* Default is 2.</li>
|
||||
* <li><b>dfs.namenode.bookkeeperjournal.digestPw</b>
|
||||
* Password to use when creating ledgers. </li>
|
||||
* <li><b>dfs.namenode.bookkeeperjournal.zk.session.timeout</b>
|
||||
* Session timeout for Zookeeper client from BookKeeper Journal Manager.
|
||||
* Hadoop recommends that, this value should be less than the ZKFC
|
||||
* session timeout value. Default value is 3000.</li>
|
||||
* </ul>
|
||||
*/
|
||||
public class BookKeeperJournalManager implements JournalManager {
|
||||
static final Log LOG = LogFactory.getLog(BookKeeperJournalManager.class);
|
||||
|
||||
public static final String BKJM_OUTPUT_BUFFER_SIZE
|
||||
= "dfs.namenode.bookkeeperjournal.output-buffer-size";
|
||||
public static final int BKJM_OUTPUT_BUFFER_SIZE_DEFAULT = 1024;
|
||||
|
||||
public static final String BKJM_BOOKKEEPER_ENSEMBLE_SIZE
|
||||
= "dfs.namenode.bookkeeperjournal.ensemble-size";
|
||||
public static final int BKJM_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT = 3;
|
||||
|
||||
public static final String BKJM_BOOKKEEPER_QUORUM_SIZE
|
||||
= "dfs.namenode.bookkeeperjournal.quorum-size";
|
||||
public static final int BKJM_BOOKKEEPER_QUORUM_SIZE_DEFAULT = 2;
|
||||
|
||||
public static final String BKJM_BOOKKEEPER_DIGEST_PW
|
||||
= "dfs.namenode.bookkeeperjournal.digestPw";
|
||||
public static final String BKJM_BOOKKEEPER_DIGEST_PW_DEFAULT = "";
|
||||
|
||||
private static final int BKJM_LAYOUT_VERSION = -1;
|
||||
|
||||
public static final String BKJM_ZK_SESSION_TIMEOUT
|
||||
= "dfs.namenode.bookkeeperjournal.zk.session.timeout";
|
||||
public static final int BKJM_ZK_SESSION_TIMEOUT_DEFAULT = 3000;
|
||||
|
||||
private static final String BKJM_EDIT_INPROGRESS = "inprogress_";
|
||||
|
||||
public static final String BKJM_ZK_LEDGERS_AVAILABLE_PATH
|
||||
= "dfs.namenode.bookkeeperjournal.zk.availablebookies";
|
||||
|
||||
public static final String BKJM_ZK_LEDGERS_AVAILABLE_PATH_DEFAULT
|
||||
= "/ledgers/available";
|
||||
|
||||
public static final String BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS
|
||||
= "dfs.namenode.bookkeeperjournal.speculativeReadTimeoutMs";
|
||||
public static final int BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_DEFAULT
|
||||
= 2000;
|
||||
|
||||
public static final String BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_SEC
|
||||
= "dfs.namenode.bookkeeperjournal.readEntryTimeoutSec";
|
||||
public static final int BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_DEFAULT = 5;
|
||||
|
||||
public static final String BKJM_BOOKKEEPER_ACK_QUORUM_SIZE
|
||||
= "dfs.namenode.bookkeeperjournal.ack.quorum-size";
|
||||
|
||||
public static final String BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC
|
||||
= "dfs.namenode.bookkeeperjournal.addEntryTimeoutSec";
|
||||
public static final int BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT = 5;
|
||||
|
||||
private ZooKeeper zkc;
|
||||
private final Configuration conf;
|
||||
private final BookKeeper bkc;
|
||||
private final CurrentInprogress ci;
|
||||
private final String basePath;
|
||||
private final String ledgerPath;
|
||||
private final String versionPath;
|
||||
private final MaxTxId maxTxId;
|
||||
private final int ensembleSize;
|
||||
private final int quorumSize;
|
||||
private final int ackQuorumSize;
|
||||
private final int addEntryTimeout;
|
||||
private final String digestpw;
|
||||
private final int speculativeReadTimeout;
|
||||
private final int readEntryTimeout;
|
||||
private final CountDownLatch zkConnectLatch;
|
||||
private final NamespaceInfo nsInfo;
|
||||
private boolean initialized = false;
|
||||
private LedgerHandle currentLedger = null;
|
||||
|
||||
/**
|
||||
* Construct a Bookkeeper journal manager.
|
||||
*/
|
||||
public BookKeeperJournalManager(Configuration conf, URI uri,
|
||||
NamespaceInfo nsInfo) throws IOException {
|
||||
this.conf = conf;
|
||||
this.nsInfo = nsInfo;
|
||||
|
||||
String zkConnect = uri.getAuthority().replace(";", ",");
|
||||
basePath = uri.getPath();
|
||||
ensembleSize = conf.getInt(BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
|
||||
BKJM_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT);
|
||||
quorumSize = conf.getInt(BKJM_BOOKKEEPER_QUORUM_SIZE,
|
||||
BKJM_BOOKKEEPER_QUORUM_SIZE_DEFAULT);
|
||||
ackQuorumSize = conf.getInt(BKJM_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
|
||||
addEntryTimeout = conf.getInt(BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
|
||||
BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_DEFAULT);
|
||||
speculativeReadTimeout = conf.getInt(
|
||||
BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS,
|
||||
BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_DEFAULT);
|
||||
readEntryTimeout = conf.getInt(BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_SEC,
|
||||
BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_DEFAULT);
|
||||
|
||||
ledgerPath = basePath + "/ledgers";
|
||||
String maxTxIdPath = basePath + "/maxtxid";
|
||||
String currentInprogressNodePath = basePath + "/CurrentInprogress";
|
||||
versionPath = basePath + "/version";
|
||||
digestpw = conf.get(BKJM_BOOKKEEPER_DIGEST_PW,
|
||||
BKJM_BOOKKEEPER_DIGEST_PW_DEFAULT);
|
||||
|
||||
try {
|
||||
zkConnectLatch = new CountDownLatch(1);
|
||||
int bkjmZKSessionTimeout = conf.getInt(BKJM_ZK_SESSION_TIMEOUT,
|
||||
BKJM_ZK_SESSION_TIMEOUT_DEFAULT);
|
||||
zkc = new ZooKeeper(zkConnect, bkjmZKSessionTimeout,
|
||||
new ZkConnectionWatcher());
|
||||
// Configured zk session timeout + some extra grace period (here
|
||||
// BKJM_ZK_SESSION_TIMEOUT_DEFAULT used as grace period)
|
||||
int zkConnectionLatchTimeout = bkjmZKSessionTimeout
|
||||
+ BKJM_ZK_SESSION_TIMEOUT_DEFAULT;
|
||||
if (!zkConnectLatch
|
||||
.await(zkConnectionLatchTimeout, TimeUnit.MILLISECONDS)) {
|
||||
throw new IOException("Error connecting to zookeeper");
|
||||
}
|
||||
|
||||
prepareBookKeeperEnv();
|
||||
ClientConfiguration clientConf = new ClientConfiguration();
|
||||
clientConf.setSpeculativeReadTimeout(speculativeReadTimeout);
|
||||
clientConf.setReadEntryTimeout(readEntryTimeout);
|
||||
clientConf.setAddEntryTimeout(addEntryTimeout);
|
||||
bkc = new BookKeeper(clientConf, zkc);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error initializing zk", e);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted while initializing bk journal manager",
|
||||
ie);
|
||||
}
|
||||
|
||||
ci = new CurrentInprogress(zkc, currentInprogressNodePath);
|
||||
maxTxId = new MaxTxId(zkc, maxTxIdPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Pre-creating bookkeeper metadata path in zookeeper.
|
||||
*/
|
||||
private void prepareBookKeeperEnv() throws IOException {
|
||||
// create bookie available path in zookeeper if it doesn't exists
|
||||
final String zkAvailablePath = conf.get(BKJM_ZK_LEDGERS_AVAILABLE_PATH,
|
||||
BKJM_ZK_LEDGERS_AVAILABLE_PATH_DEFAULT);
|
||||
final CountDownLatch zkPathLatch = new CountDownLatch(1);
|
||||
|
||||
final AtomicBoolean success = new AtomicBoolean(false);
|
||||
StringCallback callback = new StringCallback() {
|
||||
@Override
|
||||
public void processResult(int rc, String path, Object ctx, String name) {
|
||||
if (KeeperException.Code.OK.intValue() == rc
|
||||
|| KeeperException.Code.NODEEXISTS.intValue() == rc) {
|
||||
LOG.info("Successfully created bookie available path : "
|
||||
+ zkAvailablePath);
|
||||
success.set(true);
|
||||
} else {
|
||||
KeeperException.Code code = KeeperException.Code.get(rc);
|
||||
LOG.error("Error : "
|
||||
+ KeeperException.create(code, path).getMessage()
|
||||
+ ", failed to create bookie available path : "
|
||||
+ zkAvailablePath);
|
||||
}
|
||||
zkPathLatch.countDown();
|
||||
}
|
||||
};
|
||||
ZkUtils.asyncCreateFullPathOptimistic(zkc, zkAvailablePath, new byte[0],
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, callback, null);
|
||||
|
||||
try {
|
||||
if (!zkPathLatch.await(zkc.getSessionTimeout(), TimeUnit.MILLISECONDS)
|
||||
|| !success.get()) {
|
||||
throw new IOException("Couldn't create bookie available path :"
|
||||
+ zkAvailablePath + ", timed out " + zkc.getSessionTimeout()
|
||||
+ " millis");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException(
|
||||
"Interrupted when creating the bookie available path : "
|
||||
+ zkAvailablePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void format(NamespaceInfo ns) throws IOException {
|
||||
try {
|
||||
// delete old info
|
||||
Stat baseStat = null;
|
||||
Stat ledgerStat = null;
|
||||
if ((baseStat = zkc.exists(basePath, false)) != null) {
|
||||
if ((ledgerStat = zkc.exists(ledgerPath, false)) != null) {
|
||||
for (EditLogLedgerMetadata l : getLedgerList(true)) {
|
||||
try {
|
||||
bkc.deleteLedger(l.getLedgerId());
|
||||
} catch (BKException.BKNoSuchLedgerExistsException bke) {
|
||||
LOG.warn("Ledger " + l.getLedgerId() + " does not exist;"
|
||||
+ " Cannot delete.");
|
||||
}
|
||||
}
|
||||
}
|
||||
ZKUtil.deleteRecursive(zkc, basePath);
|
||||
}
|
||||
|
||||
// should be clean now.
|
||||
zkc.create(basePath, new byte[] {'0'},
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
|
||||
VersionProto.Builder builder = VersionProto.newBuilder();
|
||||
builder.setNamespaceInfo(PBHelper.convert(ns))
|
||||
.setLayoutVersion(BKJM_LAYOUT_VERSION);
|
||||
|
||||
byte[] data = TextFormat.printToString(builder.build()).getBytes(UTF_8);
|
||||
zkc.create(versionPath, data,
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
|
||||
zkc.create(ledgerPath, new byte[] {'0'},
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
} catch (KeeperException ke) {
|
||||
LOG.error("Error accessing zookeeper to format", ke);
|
||||
throw new IOException("Error accessing zookeeper to format", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted during format", ie);
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("Error cleaning up ledgers during format", bke);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasSomeData() throws IOException {
|
||||
try {
|
||||
return zkc.exists(basePath, false) != null;
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Couldn't contact zookeeper", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted while checking for data", ie);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized private void checkEnv() throws IOException {
|
||||
if (!initialized) {
|
||||
try {
|
||||
Stat versionStat = zkc.exists(versionPath, false);
|
||||
if (versionStat == null) {
|
||||
throw new IOException("Environment not initialized. "
|
||||
+"Have you forgotten to format?");
|
||||
}
|
||||
byte[] d = zkc.getData(versionPath, false, versionStat);
|
||||
|
||||
VersionProto.Builder builder = VersionProto.newBuilder();
|
||||
TextFormat.merge(new String(d, UTF_8), builder);
|
||||
if (!builder.isInitialized()) {
|
||||
throw new IOException("Invalid/Incomplete data in znode");
|
||||
}
|
||||
VersionProto vp = builder.build();
|
||||
|
||||
// There's only one version at the moment
|
||||
assert vp.getLayoutVersion() == BKJM_LAYOUT_VERSION;
|
||||
|
||||
NamespaceInfo readns = PBHelper.convert(vp.getNamespaceInfo());
|
||||
|
||||
if (nsInfo.getNamespaceID() != readns.getNamespaceID() ||
|
||||
!nsInfo.clusterID.equals(readns.getClusterID()) ||
|
||||
!nsInfo.getBlockPoolID().equals(readns.getBlockPoolID())) {
|
||||
String err = String.format("Environment mismatch. Running process %s"
|
||||
+", stored in ZK %s", nsInfo, readns);
|
||||
LOG.error(err);
|
||||
throw new IOException(err);
|
||||
}
|
||||
|
||||
ci.init();
|
||||
initialized = true;
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Cannot access ZooKeeper", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted while checking environment", ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Start a new log segment in a BookKeeper ledger.
|
||||
* First ensure that we have the write lock for this journal.
|
||||
* Then create a ledger and stream based on that ledger.
|
||||
* The ledger id is written to the inprogress znode, so that in the
|
||||
* case of a crash, a recovery process can find the ledger we were writing
|
||||
* to when we crashed.
|
||||
* @param txId First transaction id to be written to the stream
|
||||
*/
|
||||
@Override
|
||||
public EditLogOutputStream startLogSegment(long txId, int layoutVersion)
|
||||
throws IOException {
|
||||
checkEnv();
|
||||
|
||||
if (txId <= maxTxId.get()) {
|
||||
throw new IOException("We've already seen " + txId
|
||||
+ ". A new stream cannot be created with it");
|
||||
}
|
||||
|
||||
try {
|
||||
String existingInprogressNode = ci.read();
|
||||
if (null != existingInprogressNode
|
||||
&& zkc.exists(existingInprogressNode, false) != null) {
|
||||
throw new IOException("Inprogress node already exists");
|
||||
}
|
||||
if (currentLedger != null) {
|
||||
// bookkeeper errored on last stream, clean up ledger
|
||||
currentLedger.close();
|
||||
}
|
||||
currentLedger = bkc.createLedger(ensembleSize, quorumSize, ackQuorumSize,
|
||||
BookKeeper.DigestType.MAC,
|
||||
digestpw.getBytes(Charsets.UTF_8));
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("Error creating ledger", bke);
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Error in zookeeper while creating ledger", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted creating ledger", ie);
|
||||
}
|
||||
|
||||
try {
|
||||
String znodePath = inprogressZNode(txId);
|
||||
EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
|
||||
layoutVersion, currentLedger.getId(), txId);
|
||||
/* Write the ledger metadata out to the inprogress ledger znode
|
||||
* This can fail if for some reason our write lock has
|
||||
* expired (@see WriteLock) and another process has managed to
|
||||
* create the inprogress znode.
|
||||
* In this case, throw an exception. We don't want to continue
|
||||
* as this would lead to a split brain situation.
|
||||
*/
|
||||
l.write(zkc, znodePath);
|
||||
|
||||
maxTxId.store(txId);
|
||||
ci.update(znodePath);
|
||||
return new BookKeeperEditLogOutputStream(conf, currentLedger);
|
||||
} catch (KeeperException ke) {
|
||||
cleanupLedger(currentLedger);
|
||||
throw new IOException("Error storing ledger metadata", ke);
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanupLedger(LedgerHandle lh) {
|
||||
try {
|
||||
long id = currentLedger.getId();
|
||||
currentLedger.close();
|
||||
bkc.deleteLedger(id);
|
||||
} catch (BKException bke) {
|
||||
//log & ignore, an IOException will be thrown soon
|
||||
LOG.error("Error closing ledger", bke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
LOG.warn("Interrupted while closing ledger", ie);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Finalize a log segment. If the journal manager is currently
|
||||
* writing to a ledger, ensure that this is the ledger of the log segment
|
||||
* being finalized.
|
||||
*
|
||||
* Otherwise this is the recovery case. In the recovery case, ensure that
|
||||
* the firstTxId of the ledger matches firstTxId for the segment we are
|
||||
* trying to finalize.
|
||||
*/
|
||||
@Override
|
||||
public void finalizeLogSegment(long firstTxId, long lastTxId)
|
||||
throws IOException {
|
||||
checkEnv();
|
||||
|
||||
String inprogressPath = inprogressZNode(firstTxId);
|
||||
try {
|
||||
Stat inprogressStat = zkc.exists(inprogressPath, false);
|
||||
if (inprogressStat == null) {
|
||||
throw new IOException("Inprogress znode " + inprogressPath
|
||||
+ " doesn't exist");
|
||||
}
|
||||
|
||||
EditLogLedgerMetadata l
|
||||
= EditLogLedgerMetadata.read(zkc, inprogressPath);
|
||||
|
||||
if (currentLedger != null) { // normal, non-recovery case
|
||||
if (l.getLedgerId() == currentLedger.getId()) {
|
||||
try {
|
||||
currentLedger.close();
|
||||
} catch (BKException bke) {
|
||||
LOG.error("Error closing current ledger", bke);
|
||||
}
|
||||
currentLedger = null;
|
||||
} else {
|
||||
throw new IOException(
|
||||
"Active ledger has different ID to inprogress. "
|
||||
+ l.getLedgerId() + " found, "
|
||||
+ currentLedger.getId() + " expected");
|
||||
}
|
||||
}
|
||||
|
||||
if (l.getFirstTxId() != firstTxId) {
|
||||
throw new IOException("Transaction id not as expected, "
|
||||
+ l.getFirstTxId() + " found, " + firstTxId + " expected");
|
||||
}
|
||||
|
||||
l.finalizeLedger(lastTxId);
|
||||
String finalisedPath = finalizedLedgerZNode(firstTxId, lastTxId);
|
||||
try {
|
||||
l.write(zkc, finalisedPath);
|
||||
} catch (KeeperException.NodeExistsException nee) {
|
||||
if (!l.verify(zkc, finalisedPath)) {
|
||||
throw new IOException("Node " + finalisedPath + " already exists"
|
||||
+ " but data doesn't match");
|
||||
}
|
||||
}
|
||||
maxTxId.store(lastTxId);
|
||||
zkc.delete(inprogressPath, inprogressStat.getVersion());
|
||||
String inprogressPathFromCI = ci.read();
|
||||
if (inprogressPath.equals(inprogressPathFromCI)) {
|
||||
ci.clear();
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error finalising ledger", e);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Error finalising ledger", ie);
|
||||
}
|
||||
}
|
||||
|
||||
public void selectInputStreams(
|
||||
Collection<EditLogInputStream> streams,
|
||||
long fromTxnId, boolean inProgressOk) throws IOException {
|
||||
selectInputStreams(streams, fromTxnId, inProgressOk, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void selectInputStreams(Collection<EditLogInputStream> streams,
|
||||
long fromTxId, boolean inProgressOk, boolean onlyDurableTxns)
|
||||
throws IOException {
|
||||
List<EditLogLedgerMetadata> currentLedgerList = getLedgerList(fromTxId,
|
||||
inProgressOk);
|
||||
try {
|
||||
BookKeeperEditLogInputStream elis = null;
|
||||
for (EditLogLedgerMetadata l : currentLedgerList) {
|
||||
long lastTxId = l.getLastTxId();
|
||||
if (l.isInProgress()) {
|
||||
lastTxId = recoverLastTxId(l, false);
|
||||
}
|
||||
// Check once again, required in case of InProgress and is case of any
|
||||
// gap.
|
||||
if (fromTxId >= l.getFirstTxId() && fromTxId <= lastTxId) {
|
||||
LedgerHandle h;
|
||||
if (l.isInProgress()) { // we don't want to fence the current journal
|
||||
h = bkc.openLedgerNoRecovery(l.getLedgerId(),
|
||||
BookKeeper.DigestType.MAC, digestpw.getBytes(Charsets.UTF_8));
|
||||
} else {
|
||||
h = bkc.openLedger(l.getLedgerId(), BookKeeper.DigestType.MAC,
|
||||
digestpw.getBytes(Charsets.UTF_8));
|
||||
}
|
||||
elis = new BookKeeperEditLogInputStream(h, l);
|
||||
elis.skipTo(fromTxId);
|
||||
} else {
|
||||
// If mismatches then there might be some gap, so we should not check
|
||||
// further.
|
||||
return;
|
||||
}
|
||||
streams.add(elis);
|
||||
if (elis.getLastTxId() == HdfsServerConstants.INVALID_TXID) {
|
||||
return;
|
||||
}
|
||||
fromTxId = elis.getLastTxId() + 1;
|
||||
}
|
||||
} catch (BKException e) {
|
||||
throw new IOException("Could not open ledger for " + fromTxId, e);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted opening ledger for " + fromTxId, ie);
|
||||
}
|
||||
}
|
||||
|
||||
long getNumberOfTransactions(long fromTxId, boolean inProgressOk)
|
||||
throws IOException {
|
||||
long count = 0;
|
||||
long expectedStart = 0;
|
||||
for (EditLogLedgerMetadata l : getLedgerList(inProgressOk)) {
|
||||
long lastTxId = l.getLastTxId();
|
||||
if (l.isInProgress()) {
|
||||
lastTxId = recoverLastTxId(l, false);
|
||||
if (lastTxId == HdfsServerConstants.INVALID_TXID) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
assert lastTxId >= l.getFirstTxId();
|
||||
|
||||
if (lastTxId < fromTxId) {
|
||||
continue;
|
||||
} else if (l.getFirstTxId() <= fromTxId && lastTxId >= fromTxId) {
|
||||
// we can start in the middle of a segment
|
||||
count = (lastTxId - l.getFirstTxId()) + 1;
|
||||
expectedStart = lastTxId + 1;
|
||||
} else {
|
||||
if (expectedStart != l.getFirstTxId()) {
|
||||
if (count == 0) {
|
||||
throw new CorruptionException("StartTxId " + l.getFirstTxId()
|
||||
+ " is not as expected " + expectedStart
|
||||
+ ". Gap in transaction log?");
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
count += (lastTxId - l.getFirstTxId()) + 1;
|
||||
expectedStart = lastTxId + 1;
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void recoverUnfinalizedSegments() throws IOException {
|
||||
checkEnv();
|
||||
|
||||
synchronized (this) {
|
||||
try {
|
||||
List<String> children = zkc.getChildren(ledgerPath, false);
|
||||
for (String child : children) {
|
||||
if (!child.startsWith(BKJM_EDIT_INPROGRESS)) {
|
||||
continue;
|
||||
}
|
||||
String znode = ledgerPath + "/" + child;
|
||||
EditLogLedgerMetadata l = EditLogLedgerMetadata.read(zkc, znode);
|
||||
try {
|
||||
long endTxId = recoverLastTxId(l, true);
|
||||
if (endTxId == HdfsServerConstants.INVALID_TXID) {
|
||||
LOG.error("Unrecoverable corruption has occurred in segment "
|
||||
+ l.toString() + " at path " + znode
|
||||
+ ". Unable to continue recovery.");
|
||||
throw new IOException("Unrecoverable corruption,"
|
||||
+ " please check logs.");
|
||||
}
|
||||
finalizeLogSegment(l.getFirstTxId(), endTxId);
|
||||
} catch (SegmentEmptyException see) {
|
||||
LOG.warn("Inprogress znode " + child
|
||||
+ " refers to a ledger which is empty. This occurs when the NN"
|
||||
+ " crashes after opening a segment, but before writing the"
|
||||
+ " OP_START_LOG_SEGMENT op. It is safe to delete."
|
||||
+ " MetaData [" + l.toString() + "]");
|
||||
|
||||
// If the max seen transaction is the same as what would
|
||||
// have been the first transaction of the failed ledger,
|
||||
// decrement it, as that transaction never happened and as
|
||||
// such, is _not_ the last seen
|
||||
if (maxTxId.get() == l.getFirstTxId()) {
|
||||
maxTxId.reset(maxTxId.get() - 1);
|
||||
}
|
||||
|
||||
zkc.delete(znode, -1);
|
||||
}
|
||||
}
|
||||
} catch (KeeperException.NoNodeException nne) {
|
||||
// nothing to recover, ignore
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Couldn't get list of inprogress segments", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted getting list of inprogress segments",
|
||||
ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void purgeLogsOlderThan(long minTxIdToKeep)
|
||||
throws IOException {
|
||||
checkEnv();
|
||||
|
||||
for (EditLogLedgerMetadata l : getLedgerList(false)) {
|
||||
if (l.getLastTxId() < minTxIdToKeep) {
|
||||
try {
|
||||
Stat stat = zkc.exists(l.getZkPath(), false);
|
||||
zkc.delete(l.getZkPath(), stat.getVersion());
|
||||
bkc.deleteLedger(l.getLedgerId());
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
LOG.error("Interrupted while purging " + l, ie);
|
||||
} catch (BKException bke) {
|
||||
LOG.error("Couldn't delete ledger from bookkeeper", bke);
|
||||
} catch (KeeperException ke) {
|
||||
LOG.error("Error deleting ledger entry in zookeeper", ke);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doPreUpgrade() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doUpgrade(Storage storage) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getJournalCTime() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doFinalize() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
|
||||
int targetLayoutVersion) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doRollback() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void discardSegments(long startTxId) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
bkc.close();
|
||||
zkc.close();
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("Couldn't close bookkeeper client", bke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted while closing journal manager", ie);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the amount of memory that this stream should use to buffer edits.
|
||||
* Setting this will only affect future output stream. Streams
|
||||
* which have currently be created won't be affected.
|
||||
*/
|
||||
@Override
|
||||
public void setOutputBufferCapacity(int size) {
|
||||
conf.getInt(BKJM_OUTPUT_BUFFER_SIZE, size);
|
||||
}
|
||||
|
||||
/**
|
||||
* Find the id of the last edit log transaction writen to a edit log
|
||||
* ledger.
|
||||
*/
|
||||
private long recoverLastTxId(EditLogLedgerMetadata l, boolean fence)
|
||||
throws IOException, SegmentEmptyException {
|
||||
LedgerHandle lh = null;
|
||||
try {
|
||||
if (fence) {
|
||||
lh = bkc.openLedger(l.getLedgerId(),
|
||||
BookKeeper.DigestType.MAC,
|
||||
digestpw.getBytes(Charsets.UTF_8));
|
||||
} else {
|
||||
lh = bkc.openLedgerNoRecovery(l.getLedgerId(),
|
||||
BookKeeper.DigestType.MAC,
|
||||
digestpw.getBytes(Charsets.UTF_8));
|
||||
}
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("Exception opening ledger for " + l, bke);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted opening ledger for " + l, ie);
|
||||
}
|
||||
|
||||
BookKeeperEditLogInputStream in = null;
|
||||
|
||||
try {
|
||||
long lastAddConfirmed = lh.getLastAddConfirmed();
|
||||
if (lastAddConfirmed == -1) {
|
||||
throw new SegmentEmptyException();
|
||||
}
|
||||
|
||||
in = new BookKeeperEditLogInputStream(lh, l, lastAddConfirmed);
|
||||
|
||||
long endTxId = HdfsServerConstants.INVALID_TXID;
|
||||
FSEditLogOp op = in.readOp();
|
||||
while (op != null) {
|
||||
if (endTxId == HdfsServerConstants.INVALID_TXID
|
||||
|| op.getTransactionId() == endTxId+1) {
|
||||
endTxId = op.getTransactionId();
|
||||
}
|
||||
op = in.readOp();
|
||||
}
|
||||
return endTxId;
|
||||
} finally {
|
||||
if (in != null) {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of all segments in the journal.
|
||||
*/
|
||||
List<EditLogLedgerMetadata> getLedgerList(boolean inProgressOk)
|
||||
throws IOException {
|
||||
return getLedgerList(-1, inProgressOk);
|
||||
}
|
||||
|
||||
private List<EditLogLedgerMetadata> getLedgerList(long fromTxId,
|
||||
boolean inProgressOk) throws IOException {
|
||||
List<EditLogLedgerMetadata> ledgers
|
||||
= new ArrayList<EditLogLedgerMetadata>();
|
||||
try {
|
||||
List<String> ledgerNames = zkc.getChildren(ledgerPath, false);
|
||||
for (String ledgerName : ledgerNames) {
|
||||
if (!inProgressOk && ledgerName.contains(BKJM_EDIT_INPROGRESS)) {
|
||||
continue;
|
||||
}
|
||||
String legderMetadataPath = ledgerPath + "/" + ledgerName;
|
||||
try {
|
||||
EditLogLedgerMetadata editLogLedgerMetadata = EditLogLedgerMetadata
|
||||
.read(zkc, legderMetadataPath);
|
||||
if (editLogLedgerMetadata.getLastTxId() != HdfsServerConstants.INVALID_TXID
|
||||
&& editLogLedgerMetadata.getLastTxId() < fromTxId) {
|
||||
// exclude already read closed edits, but include inprogress edits
|
||||
// as this will be handled in caller
|
||||
continue;
|
||||
}
|
||||
ledgers.add(editLogLedgerMetadata);
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
LOG.warn("ZNode: " + legderMetadataPath
|
||||
+ " might have finalized and deleted."
|
||||
+ " So ignoring NoNodeException.");
|
||||
}
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Exception reading ledger list from zk", e);
|
||||
} catch (InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupted getting list of ledgers from zk", ie);
|
||||
}
|
||||
|
||||
Collections.sort(ledgers, EditLogLedgerMetadata.COMPARATOR);
|
||||
return ledgers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the znode path for a finalize ledger
|
||||
*/
|
||||
String finalizedLedgerZNode(long startTxId, long endTxId) {
|
||||
return String.format("%s/edits_%018d_%018d",
|
||||
ledgerPath, startTxId, endTxId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the znode path for the inprogressZNode
|
||||
*/
|
||||
String inprogressZNode(long startTxid) {
|
||||
return ledgerPath + "/inprogress_" + Long.toString(startTxid, 16);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void setZooKeeper(ZooKeeper zk) {
|
||||
this.zkc = zk;
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple watcher to notify when zookeeper has connected
|
||||
*/
|
||||
private class ZkConnectionWatcher implements Watcher {
|
||||
public void process(WatchedEvent event) {
|
||||
if (Event.KeeperState.SyncConnected.equals(event.getState())) {
|
||||
zkConnectLatch.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class SegmentEmptyException extends IOException {
|
||||
}
|
||||
}
|
|
@ -1,160 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.KeeperException.NodeExistsException;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.CurrentInprogressProto;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import static com.google.common.base.Charsets.UTF_8;
|
||||
|
||||
/**
|
||||
* Distributed write permission lock, using ZooKeeper. Read the version number
|
||||
* and return the current inprogress node path available in CurrentInprogress
|
||||
* path. If it exist, caller can treat that some other client already operating
|
||||
* on it. Then caller can take action. If there is no inprogress node exist,
|
||||
* then caller can treat that there is no client operating on it. Later same
|
||||
* caller should update the his newly created inprogress node path. At this
|
||||
* point, if some other activities done on this node, version number might
|
||||
* change, so update will fail. So, this read, update api will ensure that there
|
||||
* is only node can continue further after checking with CurrentInprogress.
|
||||
*/
|
||||
|
||||
class CurrentInprogress {
|
||||
static final Log LOG = LogFactory.getLog(CurrentInprogress.class);
|
||||
|
||||
private final ZooKeeper zkc;
|
||||
private final String currentInprogressNode;
|
||||
private volatile int versionNumberForPermission = -1;
|
||||
private final String hostName = InetAddress.getLocalHost().toString();
|
||||
|
||||
CurrentInprogress(ZooKeeper zkc, String lockpath) throws IOException {
|
||||
this.currentInprogressNode = lockpath;
|
||||
this.zkc = zkc;
|
||||
}
|
||||
|
||||
void init() throws IOException {
|
||||
try {
|
||||
Stat isCurrentInprogressNodeExists = zkc.exists(currentInprogressNode,
|
||||
false);
|
||||
if (isCurrentInprogressNodeExists == null) {
|
||||
try {
|
||||
zkc.create(currentInprogressNode, null, Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
} catch (NodeExistsException e) {
|
||||
// Node might created by other process at the same time. Ignore it.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(currentInprogressNode + " already created by other process.",
|
||||
e);
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Exception accessing Zookeeper", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted accessing Zookeeper", ie);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Update the path with prepending version number and hostname
|
||||
*
|
||||
* @param path
|
||||
* - to be updated in zookeeper
|
||||
* @throws IOException
|
||||
*/
|
||||
void update(String path) throws IOException {
|
||||
CurrentInprogressProto.Builder builder = CurrentInprogressProto.newBuilder();
|
||||
builder.setPath(path).setHostname(hostName);
|
||||
|
||||
String content = TextFormat.printToString(builder.build());
|
||||
|
||||
try {
|
||||
zkc.setData(this.currentInprogressNode, content.getBytes(UTF_8),
|
||||
this.versionNumberForPermission);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Exception when setting the data "
|
||||
+ "[" + content + "] to CurrentInprogress. ", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted while setting the data "
|
||||
+ "[" + content + "] to CurrentInprogress", e);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Updated data[" + content + "] to CurrentInprogress");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the CurrentInprogress node data from Zookeeper and also get the znode
|
||||
* version number. Return the 3rd field from the data. i.e saved path with
|
||||
* #update api
|
||||
*
|
||||
* @return available inprogress node path. returns null if not available.
|
||||
* @throws IOException
|
||||
*/
|
||||
String read() throws IOException {
|
||||
Stat stat = new Stat();
|
||||
byte[] data = null;
|
||||
try {
|
||||
data = zkc.getData(this.currentInprogressNode, false, stat);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Exception while reading the data from "
|
||||
+ currentInprogressNode, e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted while reading data from "
|
||||
+ currentInprogressNode, e);
|
||||
}
|
||||
this.versionNumberForPermission = stat.getVersion();
|
||||
if (data != null) {
|
||||
CurrentInprogressProto.Builder builder = CurrentInprogressProto.newBuilder();
|
||||
TextFormat.merge(new String(data, UTF_8), builder);
|
||||
if (!builder.isInitialized()) {
|
||||
throw new IOException("Invalid/Incomplete data in znode");
|
||||
}
|
||||
return builder.build().getPath();
|
||||
} else {
|
||||
LOG.debug("No data available in CurrentInprogress");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/** Clear the CurrentInprogress node data */
|
||||
void clear() throws IOException {
|
||||
try {
|
||||
zkc.setData(this.currentInprogressNode, null, versionNumberForPermission);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(
|
||||
"Exception when setting the data to CurrentInprogress node", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException(
|
||||
"Interrupted when setting the data to CurrentInprogress node", e);
|
||||
}
|
||||
LOG.debug("Cleared the data from CurrentInprogress");
|
||||
}
|
||||
|
||||
}
|
|
@ -1,217 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.EditLogLedgerProto;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import static com.google.common.base.Charsets.UTF_8;
|
||||
|
||||
/**
|
||||
* Utility class for storing the metadata associated
|
||||
* with a single edit log segment, stored in a single ledger
|
||||
*/
|
||||
public class EditLogLedgerMetadata {
|
||||
static final Log LOG = LogFactory.getLog(EditLogLedgerMetadata.class);
|
||||
|
||||
private String zkPath;
|
||||
private final int dataLayoutVersion;
|
||||
private final long ledgerId;
|
||||
private final long firstTxId;
|
||||
private long lastTxId;
|
||||
private boolean inprogress;
|
||||
|
||||
public static final Comparator COMPARATOR
|
||||
= new Comparator<EditLogLedgerMetadata>() {
|
||||
public int compare(EditLogLedgerMetadata o1,
|
||||
EditLogLedgerMetadata o2) {
|
||||
if (o1.firstTxId < o2.firstTxId) {
|
||||
return -1;
|
||||
} else if (o1.firstTxId == o2.firstTxId) {
|
||||
return 0;
|
||||
} else {
|
||||
return 1;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
EditLogLedgerMetadata(String zkPath, int dataLayoutVersion,
|
||||
long ledgerId, long firstTxId) {
|
||||
this.zkPath = zkPath;
|
||||
this.dataLayoutVersion = dataLayoutVersion;
|
||||
this.ledgerId = ledgerId;
|
||||
this.firstTxId = firstTxId;
|
||||
this.lastTxId = HdfsServerConstants.INVALID_TXID;
|
||||
this.inprogress = true;
|
||||
}
|
||||
|
||||
EditLogLedgerMetadata(String zkPath, int dataLayoutVersion,
|
||||
long ledgerId, long firstTxId,
|
||||
long lastTxId) {
|
||||
this.zkPath = zkPath;
|
||||
this.dataLayoutVersion = dataLayoutVersion;
|
||||
this.ledgerId = ledgerId;
|
||||
this.firstTxId = firstTxId;
|
||||
this.lastTxId = lastTxId;
|
||||
this.inprogress = false;
|
||||
}
|
||||
|
||||
String getZkPath() {
|
||||
return zkPath;
|
||||
}
|
||||
|
||||
long getFirstTxId() {
|
||||
return firstTxId;
|
||||
}
|
||||
|
||||
long getLastTxId() {
|
||||
return lastTxId;
|
||||
}
|
||||
|
||||
long getLedgerId() {
|
||||
return ledgerId;
|
||||
}
|
||||
|
||||
boolean isInProgress() {
|
||||
return this.inprogress;
|
||||
}
|
||||
|
||||
int getDataLayoutVersion() {
|
||||
return this.dataLayoutVersion;
|
||||
}
|
||||
|
||||
void finalizeLedger(long newLastTxId) {
|
||||
assert this.lastTxId == HdfsServerConstants.INVALID_TXID;
|
||||
this.lastTxId = newLastTxId;
|
||||
this.inprogress = false;
|
||||
}
|
||||
|
||||
static EditLogLedgerMetadata read(ZooKeeper zkc, String path)
|
||||
throws IOException, KeeperException.NoNodeException {
|
||||
try {
|
||||
byte[] data = zkc.getData(path, false, null);
|
||||
|
||||
EditLogLedgerProto.Builder builder = EditLogLedgerProto.newBuilder();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Reading " + path + " data: " + new String(data, UTF_8));
|
||||
}
|
||||
TextFormat.merge(new String(data, UTF_8), builder);
|
||||
if (!builder.isInitialized()) {
|
||||
throw new IOException("Invalid/Incomplete data in znode");
|
||||
}
|
||||
EditLogLedgerProto ledger = builder.build();
|
||||
|
||||
int dataLayoutVersion = ledger.getDataLayoutVersion();
|
||||
long ledgerId = ledger.getLedgerId();
|
||||
long firstTxId = ledger.getFirstTxId();
|
||||
if (ledger.hasLastTxId()) {
|
||||
long lastTxId = ledger.getLastTxId();
|
||||
return new EditLogLedgerMetadata(path, dataLayoutVersion,
|
||||
ledgerId, firstTxId, lastTxId);
|
||||
} else {
|
||||
return new EditLogLedgerMetadata(path, dataLayoutVersion,
|
||||
ledgerId, firstTxId);
|
||||
}
|
||||
} catch(KeeperException.NoNodeException nne) {
|
||||
throw nne;
|
||||
} catch(KeeperException ke) {
|
||||
throw new IOException("Error reading from zookeeper", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted reading from zookeeper", ie);
|
||||
}
|
||||
}
|
||||
|
||||
void write(ZooKeeper zkc, String path)
|
||||
throws IOException, KeeperException.NodeExistsException {
|
||||
this.zkPath = path;
|
||||
|
||||
EditLogLedgerProto.Builder builder = EditLogLedgerProto.newBuilder();
|
||||
builder.setDataLayoutVersion(dataLayoutVersion)
|
||||
.setLedgerId(ledgerId).setFirstTxId(firstTxId);
|
||||
|
||||
if (!inprogress) {
|
||||
builder.setLastTxId(lastTxId);
|
||||
}
|
||||
try {
|
||||
zkc.create(path, TextFormat.printToString(builder.build()).getBytes(UTF_8),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
} catch (KeeperException.NodeExistsException nee) {
|
||||
throw nee;
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error creating ledger znode", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted creating ledger znode", ie);
|
||||
}
|
||||
}
|
||||
|
||||
boolean verify(ZooKeeper zkc, String path) {
|
||||
try {
|
||||
EditLogLedgerMetadata other = read(zkc, path);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Verifying " + this.toString()
|
||||
+ " against " + other);
|
||||
}
|
||||
return other.equals(this);
|
||||
} catch (KeeperException e) {
|
||||
LOG.error("Couldn't verify data in " + path, e);
|
||||
return false;
|
||||
} catch (IOException ie) {
|
||||
LOG.error("Couldn't verify data in " + path, ie);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public boolean equals(Object o) {
|
||||
if (!(o instanceof EditLogLedgerMetadata)) {
|
||||
return false;
|
||||
}
|
||||
EditLogLedgerMetadata ol = (EditLogLedgerMetadata)o;
|
||||
return ledgerId == ol.ledgerId
|
||||
&& dataLayoutVersion == ol.dataLayoutVersion
|
||||
&& firstTxId == ol.firstTxId
|
||||
&& lastTxId == ol.lastTxId;
|
||||
}
|
||||
|
||||
public int hashCode() {
|
||||
int hash = 1;
|
||||
hash = hash * 31 + (int) ledgerId;
|
||||
hash = hash * 31 + (int) firstTxId;
|
||||
hash = hash * 31 + (int) lastTxId;
|
||||
hash = hash * 31 + dataLayoutVersion;
|
||||
return hash;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return "[LedgerId:"+ledgerId +
|
||||
", firstTxId:" + firstTxId +
|
||||
", lastTxId:" + lastTxId +
|
||||
", dataLayoutVersion:" + dataLayoutVersion + "]";
|
||||
}
|
||||
|
||||
}
|
|
@ -1,103 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import org.apache.hadoop.contrib.bkjournal.BKJournalProtos.MaxTxIdProto;
|
||||
import com.google.protobuf.TextFormat;
|
||||
import static com.google.common.base.Charsets.UTF_8;
|
||||
|
||||
/**
|
||||
* Utility class for storing and reading
|
||||
* the max seen txid in zookeeper
|
||||
*/
|
||||
class MaxTxId {
|
||||
static final Log LOG = LogFactory.getLog(MaxTxId.class);
|
||||
|
||||
private final ZooKeeper zkc;
|
||||
private final String path;
|
||||
|
||||
private Stat currentStat;
|
||||
|
||||
MaxTxId(ZooKeeper zkc, String path) {
|
||||
this.zkc = zkc;
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
synchronized void store(long maxTxId) throws IOException {
|
||||
long currentMax = get();
|
||||
if (currentMax < maxTxId) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Setting maxTxId to " + maxTxId);
|
||||
}
|
||||
reset(maxTxId);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void reset(long maxTxId) throws IOException {
|
||||
try {
|
||||
MaxTxIdProto.Builder builder = MaxTxIdProto.newBuilder().setTxId(maxTxId);
|
||||
|
||||
byte[] data = TextFormat.printToString(builder.build()).getBytes(UTF_8);
|
||||
if (currentStat != null) {
|
||||
currentStat = zkc.setData(path, data, currentStat
|
||||
.getVersion());
|
||||
} else {
|
||||
zkc.create(path, data, Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error writing max tx id", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted while writing max tx id", e);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized long get() throws IOException {
|
||||
try {
|
||||
currentStat = zkc.exists(path, false);
|
||||
if (currentStat == null) {
|
||||
return 0;
|
||||
} else {
|
||||
|
||||
byte[] bytes = zkc.getData(path, false, currentStat);
|
||||
|
||||
MaxTxIdProto.Builder builder = MaxTxIdProto.newBuilder();
|
||||
TextFormat.merge(new String(bytes, UTF_8), builder);
|
||||
if (!builder.isInitialized()) {
|
||||
throw new IOException("Invalid/Incomplete data in znode");
|
||||
}
|
||||
|
||||
return builder.build().getTxId();
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error reading the max tx id from zk", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted while reading thr max tx id", ie);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,49 +0,0 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
// This file contains protocol buffers that are used by bkjournal
|
||||
// mostly for storing data in zookeeper
|
||||
|
||||
option java_package = "org.apache.hadoop.contrib.bkjournal";
|
||||
option java_outer_classname = "BKJournalProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
package hadoop.hdfs;
|
||||
|
||||
import "hdfs.proto";
|
||||
import "HdfsServer.proto";
|
||||
|
||||
message VersionProto {
|
||||
required int32 layoutVersion = 1;
|
||||
optional NamespaceInfoProto namespaceInfo = 2;
|
||||
}
|
||||
|
||||
message EditLogLedgerProto {
|
||||
required int32 dataLayoutVersion = 1;
|
||||
required int64 ledgerId = 2;
|
||||
required int64 firstTxId = 3;
|
||||
optional int64 lastTxId = 4;
|
||||
}
|
||||
|
||||
message MaxTxIdProto {
|
||||
required int64 txId = 1;
|
||||
}
|
||||
|
||||
message CurrentInprogressProto {
|
||||
required string path = 1;
|
||||
optional string hostname = 2;
|
||||
}
|
|
@ -1,184 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.net.URI;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import org.apache.bookkeeper.proto.BookieServer;
|
||||
import org.apache.bookkeeper.conf.ServerConfiguration;
|
||||
import org.apache.bookkeeper.util.LocalBookKeeper;
|
||||
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.List;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.File;
|
||||
|
||||
/**
|
||||
* Utility class for setting up bookkeeper ensembles
|
||||
* and bringing individual bookies up and down
|
||||
*/
|
||||
class BKJMUtil {
|
||||
protected static final Log LOG = LogFactory.getLog(BKJMUtil.class);
|
||||
|
||||
int nextPort = 6000; // next port for additionally created bookies
|
||||
private Thread bkthread = null;
|
||||
private final static String zkEnsemble = "127.0.0.1:2181";
|
||||
int numBookies;
|
||||
|
||||
BKJMUtil(final int numBookies) throws Exception {
|
||||
this.numBookies = numBookies;
|
||||
|
||||
bkthread = new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
String[] args = new String[1];
|
||||
args[0] = String.valueOf(numBookies);
|
||||
LOG.info("Starting bk");
|
||||
LocalBookKeeper.main(args);
|
||||
} catch (InterruptedException e) {
|
||||
// go away quietly
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error starting local bk", e);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
void start() throws Exception {
|
||||
bkthread.start();
|
||||
if (!LocalBookKeeper.waitForServerUp(zkEnsemble, 10000)) {
|
||||
throw new Exception("Error starting zookeeper/bookkeeper");
|
||||
}
|
||||
assertEquals("Not all bookies started",
|
||||
numBookies, checkBookiesUp(numBookies, 10));
|
||||
}
|
||||
|
||||
void teardown() throws Exception {
|
||||
if (bkthread != null) {
|
||||
bkthread.interrupt();
|
||||
bkthread.join();
|
||||
}
|
||||
}
|
||||
|
||||
static ZooKeeper connectZooKeeper()
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
ZooKeeper zkc = new ZooKeeper(zkEnsemble, 3600, new Watcher() {
|
||||
public void process(WatchedEvent event) {
|
||||
if (event.getState() == Watcher.Event.KeeperState.SyncConnected) {
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
});
|
||||
if (!latch.await(3, TimeUnit.SECONDS)) {
|
||||
throw new IOException("Zookeeper took too long to connect");
|
||||
}
|
||||
return zkc;
|
||||
}
|
||||
|
||||
static URI createJournalURI(String path) throws Exception {
|
||||
return URI.create("bookkeeper://" + zkEnsemble + path);
|
||||
}
|
||||
|
||||
static void addJournalManagerDefinition(Configuration conf) {
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX + ".bookkeeper",
|
||||
"org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager");
|
||||
}
|
||||
|
||||
BookieServer newBookie() throws Exception {
|
||||
int port = nextPort++;
|
||||
ServerConfiguration bookieConf = new ServerConfiguration();
|
||||
bookieConf.setBookiePort(port);
|
||||
File tmpdir = File.createTempFile("bookie" + Integer.toString(port) + "_",
|
||||
"test");
|
||||
tmpdir.delete();
|
||||
tmpdir.mkdir();
|
||||
|
||||
bookieConf.setZkServers(zkEnsemble);
|
||||
bookieConf.setJournalDirName(tmpdir.getPath());
|
||||
bookieConf.setLedgerDirNames(new String[] { tmpdir.getPath() });
|
||||
|
||||
BookieServer b = new BookieServer(bookieConf);
|
||||
b.start();
|
||||
for (int i = 0; i < 10 && !b.isRunning(); i++) {
|
||||
Thread.sleep(10000);
|
||||
}
|
||||
if (!b.isRunning()) {
|
||||
throw new IOException("Bookie would not start");
|
||||
}
|
||||
return b;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that a number of bookies are available
|
||||
* @param count number of bookies required
|
||||
* @param timeout number of seconds to wait for bookies to start
|
||||
* @throws IOException if bookies are not started by the time the timeout hits
|
||||
*/
|
||||
int checkBookiesUp(int count, int timeout) throws Exception {
|
||||
ZooKeeper zkc = connectZooKeeper();
|
||||
try {
|
||||
int mostRecentSize = 0;
|
||||
for (int i = 0; i < timeout; i++) {
|
||||
try {
|
||||
List<String> children = zkc.getChildren("/ledgers/available",
|
||||
false);
|
||||
mostRecentSize = children.size();
|
||||
// Skip 'readonly znode' which is used for keeping R-O bookie details
|
||||
if (children.contains("readonly")) {
|
||||
mostRecentSize = children.size() - 1;
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Found " + mostRecentSize + " bookies up, "
|
||||
+ "waiting for " + count);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
for (String child : children) {
|
||||
LOG.trace(" server: " + child);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (mostRecentSize == count) {
|
||||
break;
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
// ignore
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
return mostRecentSize;
|
||||
} finally {
|
||||
zkc.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,414 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import org.junit.Test;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.AfterClass;
|
||||
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ha.ServiceFailedException;
|
||||
import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
|
||||
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
|
||||
import org.apache.hadoop.hdfs.HAUtil;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.util.ExitUtil;
|
||||
import org.apache.hadoop.util.ExitUtil.ExitException;
|
||||
|
||||
import org.apache.bookkeeper.proto.BookieServer;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* Integration test to ensure that the BookKeeper JournalManager
|
||||
* works for HDFS Namenode HA
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestBookKeeperAsHASharedDir {
|
||||
static final Log LOG = LogFactory.getLog(TestBookKeeperAsHASharedDir.class);
|
||||
|
||||
private static BKJMUtil bkutil;
|
||||
static int numBookies = 3;
|
||||
|
||||
private static final String TEST_FILE_DATA = "HA BookKeeperJournalManager";
|
||||
|
||||
@Parameters
|
||||
public static Collection<Object[]> data() {
|
||||
Collection<Object[]> params = new ArrayList<Object[]>();
|
||||
params.add(new Object[]{ Boolean.FALSE });
|
||||
params.add(new Object[]{ Boolean.TRUE });
|
||||
return params;
|
||||
}
|
||||
|
||||
private static boolean useAsyncEditLog;
|
||||
public TestBookKeeperAsHASharedDir(Boolean async) {
|
||||
useAsyncEditLog = async;
|
||||
}
|
||||
|
||||
private static Configuration getConf() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_EDITS_ASYNC_LOGGING,
|
||||
useAsyncEditLog);
|
||||
return conf;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBookkeeper() throws Exception {
|
||||
bkutil = new BKJMUtil(numBookies);
|
||||
bkutil.start();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void clearExitStatus() {
|
||||
ExitUtil.resetFirstExitException();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownBookkeeper() throws Exception {
|
||||
bkutil.teardown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test simple HA failover usecase with BK
|
||||
*/
|
||||
@Test
|
||||
public void testFailoverWithBK() throws Exception {
|
||||
MiniDFSCluster cluster = null;
|
||||
try {
|
||||
Configuration conf = getConf();
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
|
||||
BKJMUtil.createJournalURI("/hotfailover").toString());
|
||||
BKJMUtil.addJournalManagerDefinition(conf);
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(MiniDFSNNTopology.simpleHATopology())
|
||||
.numDataNodes(0)
|
||||
.manageNameDfsSharedDirs(false)
|
||||
.build();
|
||||
NameNode nn1 = cluster.getNameNode(0);
|
||||
NameNode nn2 = cluster.getNameNode(1);
|
||||
|
||||
cluster.waitActive();
|
||||
cluster.transitionToActive(0);
|
||||
|
||||
Path p = new Path("/testBKJMfailover");
|
||||
|
||||
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
|
||||
fs.mkdirs(p);
|
||||
cluster.shutdownNameNode(0);
|
||||
|
||||
cluster.transitionToActive(1);
|
||||
|
||||
assertTrue(fs.exists(p));
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test HA failover, where BK, as the shared storage, fails.
|
||||
* Once it becomes available again, a standby can come up.
|
||||
* Verify that any write happening after the BK fail is not
|
||||
* available on the standby.
|
||||
*/
|
||||
@Test
|
||||
public void testFailoverWithFailingBKCluster() throws Exception {
|
||||
int ensembleSize = numBookies + 1;
|
||||
BookieServer newBookie = bkutil.newBookie();
|
||||
assertEquals("New bookie didn't start",
|
||||
ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
|
||||
|
||||
BookieServer replacementBookie = null;
|
||||
|
||||
MiniDFSCluster cluster = null;
|
||||
|
||||
try {
|
||||
Configuration conf = getConf();
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
|
||||
BKJMUtil.createJournalURI("/hotfailoverWithFail").toString());
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
|
||||
ensembleSize);
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
|
||||
ensembleSize);
|
||||
BKJMUtil.addJournalManagerDefinition(conf);
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(MiniDFSNNTopology.simpleHATopology())
|
||||
.numDataNodes(0)
|
||||
.manageNameDfsSharedDirs(false)
|
||||
.checkExitOnShutdown(false)
|
||||
.build();
|
||||
NameNode nn1 = cluster.getNameNode(0);
|
||||
NameNode nn2 = cluster.getNameNode(1);
|
||||
|
||||
cluster.waitActive();
|
||||
cluster.transitionToActive(0);
|
||||
|
||||
Path p1 = new Path("/testBKJMFailingBKCluster1");
|
||||
Path p2 = new Path("/testBKJMFailingBKCluster2");
|
||||
|
||||
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
|
||||
fs.mkdirs(p1);
|
||||
newBookie.shutdown(); // will take down shared storage
|
||||
assertEquals("New bookie didn't stop",
|
||||
numBookies, bkutil.checkBookiesUp(numBookies, 10));
|
||||
|
||||
try {
|
||||
fs.mkdirs(p2);
|
||||
fail("mkdirs should result in the NN exiting");
|
||||
} catch (RemoteException re) {
|
||||
assertTrue(re.getClassName().contains("ExitException"));
|
||||
}
|
||||
cluster.shutdownNameNode(0);
|
||||
|
||||
try {
|
||||
cluster.transitionToActive(1);
|
||||
fail("Shouldn't have been able to transition with bookies down");
|
||||
} catch (ExitException ee) {
|
||||
assertTrue("Should shutdown due to required journal failure",
|
||||
ee.getMessage().contains(
|
||||
"starting log segment 3 failed for required journal"));
|
||||
}
|
||||
|
||||
replacementBookie = bkutil.newBookie();
|
||||
assertEquals("Replacement bookie didn't start",
|
||||
ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
|
||||
cluster.transitionToActive(1); // should work fine now
|
||||
|
||||
assertTrue(fs.exists(p1));
|
||||
assertFalse(fs.exists(p2));
|
||||
} finally {
|
||||
newBookie.shutdown();
|
||||
if (replacementBookie != null) {
|
||||
replacementBookie.shutdown();
|
||||
}
|
||||
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that two namenodes can't continue as primary
|
||||
*/
|
||||
@Test
|
||||
public void testMultiplePrimariesStarted() throws Exception {
|
||||
Path p1 = new Path("/testBKJMMultiplePrimary");
|
||||
|
||||
MiniDFSCluster cluster = null;
|
||||
try {
|
||||
Configuration conf = getConf();
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
|
||||
BKJMUtil.createJournalURI("/hotfailoverMultiple").toString());
|
||||
BKJMUtil.addJournalManagerDefinition(conf);
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(MiniDFSNNTopology.simpleHATopology())
|
||||
.numDataNodes(0)
|
||||
.manageNameDfsSharedDirs(false)
|
||||
.checkExitOnShutdown(false)
|
||||
.build();
|
||||
NameNode nn1 = cluster.getNameNode(0);
|
||||
NameNode nn2 = cluster.getNameNode(1);
|
||||
cluster.waitActive();
|
||||
cluster.transitionToActive(0);
|
||||
|
||||
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
fs.mkdirs(p1);
|
||||
nn1.getRpcServer().rollEditLog();
|
||||
cluster.transitionToActive(1);
|
||||
fs = cluster.getFileSystem(0); // get the older active server.
|
||||
|
||||
try {
|
||||
System.out.println("DMS: > *************");
|
||||
boolean foo = fs.delete(p1, true);
|
||||
System.out.println("DMS: < ************* "+foo);
|
||||
fail("Log update on older active should cause it to exit");
|
||||
} catch (RemoteException re) {
|
||||
assertTrue(re.getClassName().contains("ExitException"));
|
||||
}
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Use NameNode INTIALIZESHAREDEDITS to initialize the shared edits. i.e. copy
|
||||
* the edits log segments to new bkjm shared edits.
|
||||
*
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testInitializeBKSharedEdits() throws Exception {
|
||||
MiniDFSCluster cluster = null;
|
||||
try {
|
||||
Configuration conf = getConf();
|
||||
HAUtil.setAllowStandbyReads(conf, true);
|
||||
|
||||
MiniDFSNNTopology topology = MiniDFSNNTopology.simpleHATopology();
|
||||
cluster = new MiniDFSCluster.Builder(conf).nnTopology(topology)
|
||||
.numDataNodes(0).build();
|
||||
cluster.waitActive();
|
||||
// Shutdown and clear the current filebased shared dir.
|
||||
cluster.shutdownNameNodes();
|
||||
File shareddir = new File(cluster.getSharedEditsDir(0, 1));
|
||||
assertTrue("Initial Shared edits dir not fully deleted",
|
||||
FileUtil.fullyDelete(shareddir));
|
||||
|
||||
// Check namenodes should not start without shared dir.
|
||||
assertCanNotStartNamenode(cluster, 0);
|
||||
assertCanNotStartNamenode(cluster, 1);
|
||||
|
||||
// Configure bkjm as new shared edits dir in both namenodes
|
||||
Configuration nn1Conf = cluster.getConfiguration(0);
|
||||
Configuration nn2Conf = cluster.getConfiguration(1);
|
||||
nn1Conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
|
||||
.createJournalURI("/initializeSharedEdits").toString());
|
||||
nn2Conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
|
||||
.createJournalURI("/initializeSharedEdits").toString());
|
||||
BKJMUtil.addJournalManagerDefinition(nn1Conf);
|
||||
BKJMUtil.addJournalManagerDefinition(nn2Conf);
|
||||
|
||||
// Initialize the BKJM shared edits.
|
||||
assertFalse(NameNode.initializeSharedEdits(nn1Conf));
|
||||
|
||||
// NameNode should be able to start and should be in sync with BKJM as
|
||||
// shared dir
|
||||
assertCanStartHANameNodes(cluster, conf, "/testBKJMInitialize");
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void assertCanNotStartNamenode(MiniDFSCluster cluster, int nnIndex) {
|
||||
try {
|
||||
cluster.restartNameNode(nnIndex, false);
|
||||
fail("Should not have been able to start NN" + (nnIndex)
|
||||
+ " without shared dir");
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("Got expected exception", ioe);
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"storage directory does not exist or is not accessible", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertCanStartHANameNodes(MiniDFSCluster cluster,
|
||||
Configuration conf, String path) throws ServiceFailedException,
|
||||
IOException, URISyntaxException, InterruptedException {
|
||||
// Now should be able to start both NNs. Pass "false" here so that we don't
|
||||
// try to waitActive on all NNs, since the second NN doesn't exist yet.
|
||||
cluster.restartNameNode(0, false);
|
||||
cluster.restartNameNode(1, true);
|
||||
|
||||
// Make sure HA is working.
|
||||
cluster
|
||||
.getNameNode(0)
|
||||
.getRpcServer()
|
||||
.transitionToActive(
|
||||
new StateChangeRequestInfo(RequestSource.REQUEST_BY_USER));
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
Path newPath = new Path(path);
|
||||
fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
assertTrue(fs.mkdirs(newPath));
|
||||
HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
|
||||
cluster.getNameNode(1));
|
||||
assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
|
||||
newPath.toString(), false).isDir());
|
||||
} finally {
|
||||
if (fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* NameNode should load the edits correctly if the applicable edits are
|
||||
* present in the BKJM.
|
||||
*/
|
||||
@Test
|
||||
public void testNameNodeMultipleSwitchesUsingBKJM() throws Exception {
|
||||
MiniDFSCluster cluster = null;
|
||||
try {
|
||||
Configuration conf = getConf();
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
|
||||
.createJournalURI("/correctEditLogSelection").toString());
|
||||
BKJMUtil.addJournalManagerDefinition(conf);
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
|
||||
.manageNameDfsSharedDirs(false).build();
|
||||
NameNode nn1 = cluster.getNameNode(0);
|
||||
NameNode nn2 = cluster.getNameNode(1);
|
||||
cluster.waitActive();
|
||||
cluster.transitionToActive(0);
|
||||
nn1.getRpcServer().rollEditLog(); // Roll Edits from current Active.
|
||||
// Transition to standby current active gracefully.
|
||||
cluster.transitionToStandby(0);
|
||||
// Make the other Active and Roll edits multiple times
|
||||
cluster.transitionToActive(1);
|
||||
nn2.getRpcServer().rollEditLog();
|
||||
nn2.getRpcServer().rollEditLog();
|
||||
// Now One more failover. So NN1 should be able to failover successfully.
|
||||
cluster.transitionToStandby(1);
|
||||
cluster.transitionToActive(0);
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,174 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.bookkeeper.util.LocalBookKeeper;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.ZKUtil;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.server.NIOServerCnxnFactory;
|
||||
import org.apache.zookeeper.server.ZooKeeperServer;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||
|
||||
public class TestBookKeeperConfiguration {
|
||||
private static final Log LOG = LogFactory
|
||||
.getLog(TestBookKeeperConfiguration.class);
|
||||
private static final int ZK_SESSION_TIMEOUT = 5000;
|
||||
private static final String HOSTPORT = "127.0.0.1:2181";
|
||||
private static final int CONNECTION_TIMEOUT = 30000;
|
||||
private static NIOServerCnxnFactory serverFactory;
|
||||
private static ZooKeeperServer zks;
|
||||
private static ZooKeeper zkc;
|
||||
private static int ZooKeeperDefaultPort = 2181;
|
||||
private static File ZkTmpDir;
|
||||
private BookKeeperJournalManager bkjm;
|
||||
private static final String BK_ROOT_PATH = "/ledgers";
|
||||
|
||||
private static ZooKeeper connectZooKeeper(String ensemble)
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
ZooKeeper zkc = new ZooKeeper(HOSTPORT, ZK_SESSION_TIMEOUT, new Watcher() {
|
||||
public void process(WatchedEvent event) {
|
||||
if (event.getState() == Watcher.Event.KeeperState.SyncConnected) {
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
});
|
||||
if (!latch.await(ZK_SESSION_TIMEOUT, TimeUnit.MILLISECONDS)) {
|
||||
throw new IOException("Zookeeper took too long to connect");
|
||||
}
|
||||
return zkc;
|
||||
}
|
||||
|
||||
private NamespaceInfo newNSInfo() {
|
||||
Random r = new Random();
|
||||
return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupZooKeeper() throws Exception {
|
||||
// create a ZooKeeper server(dataDir, dataLogDir, port)
|
||||
LOG.info("Starting ZK server");
|
||||
ZkTmpDir = File.createTempFile("zookeeper", "test");
|
||||
ZkTmpDir.delete();
|
||||
ZkTmpDir.mkdir();
|
||||
|
||||
try {
|
||||
zks = new ZooKeeperServer(ZkTmpDir, ZkTmpDir, ZooKeeperDefaultPort);
|
||||
serverFactory = new NIOServerCnxnFactory();
|
||||
serverFactory.configure(new InetSocketAddress(ZooKeeperDefaultPort), 10);
|
||||
serverFactory.startup(zks);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception while instantiating ZooKeeper", e);
|
||||
}
|
||||
|
||||
boolean b = LocalBookKeeper.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT);
|
||||
LOG.debug("ZooKeeper server up: " + b);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
zkc = connectZooKeeper(HOSTPORT);
|
||||
try {
|
||||
ZKUtil.deleteRecursive(zkc, BK_ROOT_PATH);
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
LOG.debug("Ignoring no node exception on cleanup", e);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception when deleting bookie root path in zk", e);
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws Exception {
|
||||
if (null != zkc) {
|
||||
zkc.close();
|
||||
}
|
||||
if (null != bkjm) {
|
||||
bkjm.close();
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownZooKeeper() throws Exception {
|
||||
if (null != zkc) {
|
||||
zkc.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify the BKJM is creating the bookie available path configured in
|
||||
* 'dfs.namenode.bookkeeperjournal.zk.availablebookies'
|
||||
*/
|
||||
@Test
|
||||
public void testWithConfiguringBKAvailablePath() throws Exception {
|
||||
// set Bookie available path in the configuration
|
||||
String bkAvailablePath
|
||||
= BookKeeperJournalManager.BKJM_ZK_LEDGERS_AVAILABLE_PATH_DEFAULT;
|
||||
Configuration conf = new Configuration();
|
||||
conf.setStrings(BookKeeperJournalManager.BKJM_ZK_LEDGERS_AVAILABLE_PATH,
|
||||
bkAvailablePath);
|
||||
Assert.assertNull(bkAvailablePath + " already exists", zkc.exists(
|
||||
bkAvailablePath, false));
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
bkjm = new BookKeeperJournalManager(conf,
|
||||
URI.create("bookkeeper://" + HOSTPORT + "/hdfsjournal-WithBKPath"),
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
Assert.assertNotNull("Bookie available path : " + bkAvailablePath
|
||||
+ " doesn't exists", zkc.exists(bkAvailablePath, false));
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify the BKJM is creating the bookie available default path, when there
|
||||
* is no 'dfs.namenode.bookkeeperjournal.zk.availablebookies' configured
|
||||
*/
|
||||
@Test
|
||||
public void testDefaultBKAvailablePath() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
Assert.assertNull(BK_ROOT_PATH + " already exists", zkc.exists(
|
||||
BK_ROOT_PATH, false));
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
bkjm = new BookKeeperJournalManager(conf,
|
||||
URI.create("bookkeeper://" + HOSTPORT + "/hdfsjournal-DefaultBKPath"),
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
Assert.assertNotNull("Bookie available path : " + BK_ROOT_PATH
|
||||
+ " doesn't exists", zkc.exists(BK_ROOT_PATH, false));
|
||||
}
|
||||
}
|
|
@ -1,92 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.bookkeeper.client.BookKeeper;
|
||||
import org.apache.bookkeeper.client.LedgerHandle;
|
||||
import org.apache.bookkeeper.conf.ClientConfiguration;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Unit test for the bkjm's streams
|
||||
*/
|
||||
public class TestBookKeeperEditLogStreams {
|
||||
static final Log LOG = LogFactory.getLog(TestBookKeeperEditLogStreams.class);
|
||||
|
||||
private static BKJMUtil bkutil;
|
||||
private final static int numBookies = 3;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBookkeeper() throws Exception {
|
||||
bkutil = new BKJMUtil(numBookies);
|
||||
bkutil.start();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownBookkeeper() throws Exception {
|
||||
bkutil.teardown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that bkjm will refuse open a stream on an empty
|
||||
* ledger.
|
||||
*/
|
||||
@Test
|
||||
public void testEmptyInputStream() throws Exception {
|
||||
ZooKeeper zk = BKJMUtil.connectZooKeeper();
|
||||
|
||||
BookKeeper bkc = new BookKeeper(new ClientConfiguration(), zk);
|
||||
try {
|
||||
LedgerHandle lh = bkc.createLedger(BookKeeper.DigestType.CRC32, "foobar"
|
||||
.getBytes());
|
||||
lh.close();
|
||||
|
||||
EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar",
|
||||
HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
|
||||
try {
|
||||
new BookKeeperEditLogInputStream(lh, metadata, -1);
|
||||
fail("Shouldn't get this far, should have thrown");
|
||||
} catch (IOException ioe) {
|
||||
assertTrue(ioe.getMessage().contains("Invalid first bk entry to read"));
|
||||
}
|
||||
|
||||
metadata = new EditLogLedgerMetadata("/foobar",
|
||||
HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
|
||||
try {
|
||||
new BookKeeperEditLogInputStream(lh, metadata, 0);
|
||||
fail("Shouldn't get this far, should have thrown");
|
||||
} catch (IOException ioe) {
|
||||
assertTrue(ioe.getMessage().contains("Invalid first bk entry to read"));
|
||||
}
|
||||
} finally {
|
||||
bkc.close();
|
||||
zk.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,109 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.net.BindException;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
* Runs the same tests as TestStandbyCheckpoints, but
|
||||
* using a bookkeeper journal manager as the shared directory
|
||||
*/
|
||||
public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
|
||||
//overwrite the nn count
|
||||
static{
|
||||
TestStandbyCheckpoints.NUM_NNS = 2;
|
||||
}
|
||||
private static BKJMUtil bkutil = null;
|
||||
static int numBookies = 3;
|
||||
static int journalCount = 0;
|
||||
private final Random random = new Random();
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
@Before
|
||||
public void setupCluster() throws Exception {
|
||||
Configuration conf = setupCommonConfig();
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
|
||||
BKJMUtil.createJournalURI("/checkpointing" + journalCount++)
|
||||
.toString());
|
||||
BKJMUtil.addJournalManagerDefinition(conf);
|
||||
|
||||
int retryCount = 0;
|
||||
while (true) {
|
||||
try {
|
||||
int basePort = 10060 + random.nextInt(100) * 2;
|
||||
MiniDFSNNTopology topology = new MiniDFSNNTopology()
|
||||
.addNameservice(new MiniDFSNNTopology.NSConf("ns1")
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(topology)
|
||||
.numDataNodes(1)
|
||||
.manageNameDfsSharedDirs(false)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
|
||||
setNNs();
|
||||
fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
|
||||
cluster.transitionToActive(0);
|
||||
++retryCount;
|
||||
break;
|
||||
} catch (BindException e) {
|
||||
LOG.info("Set up MiniDFSCluster failed due to port conflicts, retry "
|
||||
+ retryCount + " times");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void startBK() throws Exception {
|
||||
journalCount = 0;
|
||||
bkutil = new BKJMUtil(numBookies);
|
||||
bkutil.start();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void shutdownBK() throws Exception {
|
||||
if (bkutil != null) {
|
||||
bkutil.teardown();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testCheckpointCancellation() throws Exception {
|
||||
// Overriden as the implementation in the superclass assumes that writes
|
||||
// are to a file. This should be fixed at some point
|
||||
}
|
||||
}
|
|
@ -1,984 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import org.junit.Test;
|
||||
import org.junit.Before;
|
||||
import org.junit.After;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.AfterClass;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.namenode.JournalManager;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||
|
||||
import org.apache.bookkeeper.proto.BookieServer;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
public class TestBookKeeperJournalManager {
|
||||
static final Log LOG = LogFactory.getLog(TestBookKeeperJournalManager.class);
|
||||
|
||||
private static final long DEFAULT_SEGMENT_SIZE = 1000;
|
||||
|
||||
protected static Configuration conf = new Configuration();
|
||||
private ZooKeeper zkc;
|
||||
private static BKJMUtil bkutil;
|
||||
static int numBookies = 3;
|
||||
private BookieServer newBookie;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBookkeeper() throws Exception {
|
||||
bkutil = new BKJMUtil(numBookies);
|
||||
bkutil.start();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownBookkeeper() throws Exception {
|
||||
bkutil.teardown();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
zkc = BKJMUtil.connectZooKeeper();
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws Exception {
|
||||
zkc.close();
|
||||
if (newBookie != null) {
|
||||
newBookie.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
private NamespaceInfo newNSInfo() {
|
||||
Random r = new Random();
|
||||
return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleWrite() throws Exception {
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-simplewrite"), nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long i = 1 ; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, 100);
|
||||
|
||||
String zkpath = bkjm.finalizedLedgerZNode(1, 100);
|
||||
|
||||
assertNotNull(zkc.exists(zkpath, false));
|
||||
assertNull(zkc.exists(bkjm.inprogressZNode(1), false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNumberOfTransactions() throws Exception {
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-txncount"), nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long i = 1 ; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, 100);
|
||||
|
||||
long numTrans = bkjm.getNumberOfTransactions(1, true);
|
||||
assertEquals(100, numTrans);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNumberOfTransactionsWithGaps() throws Exception {
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-gaps"), nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
long txid = 1;
|
||||
for (long i = 0; i < 3; i++) {
|
||||
long start = txid;
|
||||
EditLogOutputStream out = bkjm.startLogSegment(start,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(start, txid-1);
|
||||
assertNotNull(
|
||||
zkc.exists(bkjm.finalizedLedgerZNode(start, txid-1), false));
|
||||
}
|
||||
zkc.delete(bkjm.finalizedLedgerZNode(DEFAULT_SEGMENT_SIZE+1,
|
||||
DEFAULT_SEGMENT_SIZE*2), -1);
|
||||
|
||||
long numTrans = bkjm.getNumberOfTransactions(1, true);
|
||||
assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
|
||||
|
||||
try {
|
||||
numTrans = bkjm.getNumberOfTransactions(DEFAULT_SEGMENT_SIZE+1, true);
|
||||
fail("Should have thrown corruption exception by this point");
|
||||
} catch (JournalManager.CorruptionException ce) {
|
||||
// if we get here, everything is going good
|
||||
}
|
||||
|
||||
numTrans = bkjm.getNumberOfTransactions((DEFAULT_SEGMENT_SIZE*2)+1, true);
|
||||
assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNumberOfTransactionsWithInprogressAtEnd() throws Exception {
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-inprogressAtEnd"), nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
long txid = 1;
|
||||
for (long i = 0; i < 3; i++) {
|
||||
long start = txid;
|
||||
EditLogOutputStream out = bkjm.startLogSegment(start,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(start, (txid-1));
|
||||
assertNotNull(
|
||||
zkc.exists(bkjm.finalizedLedgerZNode(start, (txid-1)), false));
|
||||
}
|
||||
long start = txid;
|
||||
EditLogOutputStream out = bkjm.startLogSegment(start,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE/2; j++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.setReadyToFlush();
|
||||
out.flush();
|
||||
out.abort();
|
||||
out.close();
|
||||
|
||||
long numTrans = bkjm.getNumberOfTransactions(1, true);
|
||||
assertEquals((txid-1), numTrans);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a bkjm namespace, write a journal from txid 1, close stream.
|
||||
* Try to create a new journal from txid 1. Should throw an exception.
|
||||
*/
|
||||
@Test
|
||||
public void testWriteRestartFrom1() throws Exception {
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-restartFrom1"), nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
long txid = 1;
|
||||
long start = txid;
|
||||
EditLogOutputStream out = bkjm.startLogSegment(txid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(start, (txid-1));
|
||||
|
||||
txid = 1;
|
||||
try {
|
||||
out = bkjm.startLogSegment(txid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
fail("Shouldn't be able to start another journal from " + txid
|
||||
+ " when one already exists");
|
||||
} catch (Exception ioe) {
|
||||
LOG.info("Caught exception as expected", ioe);
|
||||
}
|
||||
|
||||
// test border case
|
||||
txid = DEFAULT_SEGMENT_SIZE;
|
||||
try {
|
||||
out = bkjm.startLogSegment(txid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
fail("Shouldn't be able to start another journal from " + txid
|
||||
+ " when one already exists");
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("Caught exception as expected", ioe);
|
||||
}
|
||||
|
||||
// open journal continuing from before
|
||||
txid = DEFAULT_SEGMENT_SIZE + 1;
|
||||
start = txid;
|
||||
out = bkjm.startLogSegment(start,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
assertNotNull(out);
|
||||
|
||||
for (long j = 1 ; j <= DEFAULT_SEGMENT_SIZE; j++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(start, (txid-1));
|
||||
|
||||
// open journal arbitarily far in the future
|
||||
txid = DEFAULT_SEGMENT_SIZE * 4;
|
||||
out = bkjm.startLogSegment(txid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
assertNotNull(out);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTwoWriters() throws Exception {
|
||||
long start = 1;
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
|
||||
BookKeeperJournalManager bkjm1 = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"), nsi);
|
||||
bkjm1.format(nsi);
|
||||
|
||||
BookKeeperJournalManager bkjm2 = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-dualWriter"), nsi);
|
||||
|
||||
|
||||
EditLogOutputStream out1 = bkjm1.startLogSegment(start,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
try {
|
||||
bkjm2.startLogSegment(start,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
fail("Shouldn't have been able to open the second writer");
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("Caught exception as expected", ioe);
|
||||
}finally{
|
||||
out1.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleRead() throws Exception {
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-simpleread"),
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
final long numTransactions = 10000;
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
|
||||
for (long i = 1 ; i <= numTransactions; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, numTransactions);
|
||||
|
||||
List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
|
||||
bkjm.selectInputStreams(in, 1, true);
|
||||
try {
|
||||
assertEquals(numTransactions,
|
||||
FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
|
||||
} finally {
|
||||
in.get(0).close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleRecovery() throws Exception {
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-simplerecovery"),
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
|
||||
for (long i = 1 ; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.setReadyToFlush();
|
||||
out.flush();
|
||||
|
||||
out.abort();
|
||||
out.close();
|
||||
|
||||
|
||||
assertNull(zkc.exists(bkjm.finalizedLedgerZNode(1, 100), false));
|
||||
assertNotNull(zkc.exists(bkjm.inprogressZNode(1), false));
|
||||
|
||||
bkjm.recoverUnfinalizedSegments();
|
||||
|
||||
assertNotNull(zkc.exists(bkjm.finalizedLedgerZNode(1, 100), false));
|
||||
assertNull(zkc.exists(bkjm.inprogressZNode(1), false));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that if enough bookies fail to prevent an ensemble,
|
||||
* writes the bookkeeper will fail. Test that when once again
|
||||
* an ensemble is available, it can continue to write.
|
||||
*/
|
||||
@Test
|
||||
public void testAllBookieFailure() throws Exception {
|
||||
// bookie to fail
|
||||
newBookie = bkutil.newBookie();
|
||||
BookieServer replacementBookie = null;
|
||||
|
||||
try {
|
||||
int ensembleSize = numBookies + 1;
|
||||
assertEquals("New bookie didn't start",
|
||||
ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
|
||||
|
||||
// ensure that the journal manager has to use all bookies,
|
||||
// so that a failure will fail the journal manager
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
|
||||
ensembleSize);
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
|
||||
ensembleSize);
|
||||
long txid = 1;
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-allbookiefailure"),
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
EditLogOutputStream out = bkjm.startLogSegment(txid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
|
||||
for (long i = 1 ; i <= 3; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.setReadyToFlush();
|
||||
out.flush();
|
||||
newBookie.shutdown();
|
||||
assertEquals("New bookie didn't die",
|
||||
numBookies, bkutil.checkBookiesUp(numBookies, 10));
|
||||
|
||||
try {
|
||||
for (long i = 1 ; i <= 3; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.setReadyToFlush();
|
||||
out.flush();
|
||||
fail("should not get to this stage");
|
||||
} catch (IOException ioe) {
|
||||
LOG.debug("Error writing to bookkeeper", ioe);
|
||||
assertTrue("Invalid exception message",
|
||||
ioe.getMessage().contains("Failed to write to bookkeeper"));
|
||||
}
|
||||
replacementBookie = bkutil.newBookie();
|
||||
|
||||
assertEquals("New bookie didn't start",
|
||||
numBookies+1, bkutil.checkBookiesUp(numBookies+1, 10));
|
||||
bkjm.recoverUnfinalizedSegments();
|
||||
out = bkjm.startLogSegment(txid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long i = 1 ; i <= 3; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
|
||||
out.setReadyToFlush();
|
||||
out.flush();
|
||||
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception in test", e);
|
||||
throw e;
|
||||
} finally {
|
||||
if (replacementBookie != null) {
|
||||
replacementBookie.shutdown();
|
||||
}
|
||||
newBookie.shutdown();
|
||||
|
||||
if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
|
||||
LOG.warn("Not all bookies from this test shut down, expect errors");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that a BookKeeper JM can continue to work across the
|
||||
* failure of a bookie. This should be handled transparently
|
||||
* by bookkeeper.
|
||||
*/
|
||||
@Test
|
||||
public void testOneBookieFailure() throws Exception {
|
||||
newBookie = bkutil.newBookie();
|
||||
BookieServer replacementBookie = null;
|
||||
|
||||
try {
|
||||
int ensembleSize = numBookies + 1;
|
||||
assertEquals("New bookie didn't start",
|
||||
ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
|
||||
|
||||
// ensure that the journal manager has to use all bookies,
|
||||
// so that a failure will fail the journal manager
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
|
||||
ensembleSize);
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
|
||||
ensembleSize);
|
||||
long txid = 1;
|
||||
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"),
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(txid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long i = 1 ; i <= 3; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.setReadyToFlush();
|
||||
out.flush();
|
||||
|
||||
replacementBookie = bkutil.newBookie();
|
||||
assertEquals("replacement bookie didn't start",
|
||||
ensembleSize+1, bkutil.checkBookiesUp(ensembleSize+1, 10));
|
||||
newBookie.shutdown();
|
||||
assertEquals("New bookie didn't die",
|
||||
ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
|
||||
|
||||
for (long i = 1 ; i <= 3; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(txid++);
|
||||
out.write(op);
|
||||
}
|
||||
out.setReadyToFlush();
|
||||
out.flush();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception in test", e);
|
||||
throw e;
|
||||
} finally {
|
||||
if (replacementBookie != null) {
|
||||
replacementBookie.shutdown();
|
||||
}
|
||||
newBookie.shutdown();
|
||||
|
||||
if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
|
||||
LOG.warn("Not all bookies from this test shut down, expect errors");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If a journal manager has an empty inprogress node, ensure that we throw an
|
||||
* error, as this should not be possible, and some third party has corrupted
|
||||
* the zookeeper state
|
||||
*/
|
||||
@Test
|
||||
public void testEmptyInprogressNode() throws Exception {
|
||||
URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogress");
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
|
||||
for (long i = 1; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, 100);
|
||||
|
||||
out = bkjm.startLogSegment(101,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
out.close();
|
||||
bkjm.close();
|
||||
String inprogressZNode = bkjm.inprogressZNode(101);
|
||||
zkc.setData(inprogressZNode, new byte[0], -1);
|
||||
|
||||
bkjm = new BookKeeperJournalManager(conf, uri, nsi);
|
||||
try {
|
||||
bkjm.recoverUnfinalizedSegments();
|
||||
fail("Should have failed. There should be no way of creating"
|
||||
+ " an empty inprogess znode");
|
||||
} catch (IOException e) {
|
||||
// correct behaviour
|
||||
assertTrue("Exception different than expected", e.getMessage().contains(
|
||||
"Invalid/Incomplete data in znode"));
|
||||
} finally {
|
||||
bkjm.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If a journal manager has an corrupt inprogress node, ensure that we throw
|
||||
* an error, as this should not be possible, and some third party has
|
||||
* corrupted the zookeeper state
|
||||
*/
|
||||
@Test
|
||||
public void testCorruptInprogressNode() throws Exception {
|
||||
URI uri = BKJMUtil.createJournalURI("/hdfsjournal-corruptInprogress");
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
|
||||
for (long i = 1; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, 100);
|
||||
|
||||
out = bkjm.startLogSegment(101,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
out.close();
|
||||
bkjm.close();
|
||||
|
||||
String inprogressZNode = bkjm.inprogressZNode(101);
|
||||
zkc.setData(inprogressZNode, "WholeLottaJunk".getBytes(), -1);
|
||||
|
||||
bkjm = new BookKeeperJournalManager(conf, uri, nsi);
|
||||
try {
|
||||
bkjm.recoverUnfinalizedSegments();
|
||||
fail("Should have failed. There should be no way of creating"
|
||||
+ " an empty inprogess znode");
|
||||
} catch (IOException e) {
|
||||
// correct behaviour
|
||||
assertTrue("Exception different than expected", e.getMessage().contains(
|
||||
"has no field named"));
|
||||
} finally {
|
||||
bkjm.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Cases can occur where we create a segment but crash before we even have the
|
||||
* chance to write the START_SEGMENT op. If this occurs we should warn, but
|
||||
* load as normal
|
||||
*/
|
||||
@Test
|
||||
public void testEmptyInprogressLedger() throws Exception {
|
||||
URI uri = BKJMUtil.createJournalURI("/hdfsjournal-emptyInprogressLedger");
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
|
||||
for (long i = 1; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, 100);
|
||||
|
||||
out = bkjm.startLogSegment(101,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
out.close();
|
||||
bkjm.close();
|
||||
|
||||
bkjm = new BookKeeperJournalManager(conf, uri, nsi);
|
||||
bkjm.recoverUnfinalizedSegments();
|
||||
out = bkjm.startLogSegment(101,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long i = 1; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(101, 200);
|
||||
|
||||
bkjm.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that if we fail between finalizing an inprogress and deleting the
|
||||
* corresponding inprogress znode.
|
||||
*/
|
||||
@Test
|
||||
public void testRefinalizeAlreadyFinalizedInprogress() throws Exception {
|
||||
URI uri = BKJMUtil
|
||||
.createJournalURI("/hdfsjournal-refinalizeInprogressLedger");
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);;
|
||||
for (long i = 1; i <= 100; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.close();
|
||||
|
||||
String inprogressZNode = bkjm.inprogressZNode(1);
|
||||
String finalizedZNode = bkjm.finalizedLedgerZNode(1, 100);
|
||||
assertNotNull("inprogress znode doesn't exist", zkc.exists(inprogressZNode,
|
||||
null));
|
||||
assertNull("finalized znode exists", zkc.exists(finalizedZNode, null));
|
||||
|
||||
byte[] inprogressData = zkc.getData(inprogressZNode, false, null);
|
||||
|
||||
// finalize
|
||||
bkjm = new BookKeeperJournalManager(conf, uri, nsi);
|
||||
bkjm.recoverUnfinalizedSegments();
|
||||
bkjm.close();
|
||||
|
||||
assertNull("inprogress znode exists", zkc.exists(inprogressZNode, null));
|
||||
assertNotNull("finalized znode doesn't exist", zkc.exists(finalizedZNode,
|
||||
null));
|
||||
|
||||
zkc.create(inprogressZNode, inprogressData, Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
// should work fine
|
||||
bkjm = new BookKeeperJournalManager(conf, uri, nsi);
|
||||
bkjm.recoverUnfinalizedSegments();
|
||||
bkjm.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that the edit log file meta data reading from ZooKeeper should be
|
||||
* able to handle the NoNodeException. bkjm.getInputStream(fromTxId,
|
||||
* inProgressOk) should suppress the NoNodeException and continue. HDFS-3441.
|
||||
*/
|
||||
@Test
|
||||
public void testEditLogFileNotExistsWhenReadingMetadata() throws Exception {
|
||||
URI uri = BKJMUtil.createJournalURI("/hdfsjournal-editlogfile");
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf, uri,
|
||||
nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
try {
|
||||
// start new inprogress log segment with txid=1
|
||||
// and write transactions till txid=50
|
||||
String zkpath1 = startAndFinalizeLogSegment(bkjm, 1, 50);
|
||||
|
||||
// start new inprogress log segment with txid=51
|
||||
// and write transactions till txid=100
|
||||
String zkpath2 = startAndFinalizeLogSegment(bkjm, 51, 100);
|
||||
|
||||
// read the metadata from ZK. Here simulating the situation
|
||||
// when reading,the edit log metadata can be removed by purger thread.
|
||||
ZooKeeper zkspy = spy(BKJMUtil.connectZooKeeper());
|
||||
bkjm.setZooKeeper(zkspy);
|
||||
Mockito.doThrow(
|
||||
new KeeperException.NoNodeException(zkpath2 + " doesn't exists"))
|
||||
.when(zkspy).getData(zkpath2, false, null);
|
||||
|
||||
List<EditLogLedgerMetadata> ledgerList = bkjm.getLedgerList(false);
|
||||
assertEquals("List contains the metadata of non exists path.", 1,
|
||||
ledgerList.size());
|
||||
assertEquals("LogLedgerMetadata contains wrong zk paths.", zkpath1,
|
||||
ledgerList.get(0).getZkPath());
|
||||
} finally {
|
||||
bkjm.close();
|
||||
}
|
||||
}
|
||||
|
||||
private enum ThreadStatus {
|
||||
COMPLETED, GOODEXCEPTION, BADEXCEPTION;
|
||||
};
|
||||
|
||||
/**
|
||||
* Tests that concurrent calls to format will still allow one to succeed.
|
||||
*/
|
||||
@Test
|
||||
public void testConcurrentFormat() throws Exception {
|
||||
final URI uri = BKJMUtil.createJournalURI("/hdfsjournal-concurrentformat");
|
||||
final NamespaceInfo nsi = newNSInfo();
|
||||
|
||||
// populate with data first
|
||||
BookKeeperJournalManager bkjm
|
||||
= new BookKeeperJournalManager(conf, uri, nsi);
|
||||
bkjm.format(nsi);
|
||||
for (int i = 1; i < 100*2; i += 2) {
|
||||
bkjm.startLogSegment(i, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
bkjm.finalizeLogSegment(i, i+1);
|
||||
}
|
||||
bkjm.close();
|
||||
|
||||
final int numThreads = 40;
|
||||
List<Callable<ThreadStatus>> threads
|
||||
= new ArrayList<Callable<ThreadStatus>>();
|
||||
final CyclicBarrier barrier = new CyclicBarrier(numThreads);
|
||||
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
threads.add(new Callable<ThreadStatus>() {
|
||||
public ThreadStatus call() {
|
||||
BookKeeperJournalManager bkjm = null;
|
||||
try {
|
||||
bkjm = new BookKeeperJournalManager(conf, uri, nsi);
|
||||
barrier.await();
|
||||
bkjm.format(nsi);
|
||||
return ThreadStatus.COMPLETED;
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("Exception formatting ", ioe);
|
||||
return ThreadStatus.GOODEXCEPTION;
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Interrupted. Something is broken", ie);
|
||||
Thread.currentThread().interrupt();
|
||||
return ThreadStatus.BADEXCEPTION;
|
||||
} catch (Exception e) {
|
||||
LOG.error("Some other bad exception", e);
|
||||
return ThreadStatus.BADEXCEPTION;
|
||||
} finally {
|
||||
if (bkjm != null) {
|
||||
try {
|
||||
bkjm.close();
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("Error closing journal manager", ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
ExecutorService service = Executors.newFixedThreadPool(numThreads);
|
||||
List<Future<ThreadStatus>> statuses = service.invokeAll(threads, 60,
|
||||
TimeUnit.SECONDS);
|
||||
int numCompleted = 0;
|
||||
for (Future<ThreadStatus> s : statuses) {
|
||||
assertTrue(s.isDone());
|
||||
assertTrue("Thread threw invalid exception",
|
||||
s.get() == ThreadStatus.COMPLETED
|
||||
|| s.get() == ThreadStatus.GOODEXCEPTION);
|
||||
if (s.get() == ThreadStatus.COMPLETED) {
|
||||
numCompleted++;
|
||||
}
|
||||
}
|
||||
LOG.info("Completed " + numCompleted + " formats");
|
||||
assertTrue("No thread managed to complete formatting", numCompleted > 0);
|
||||
}
|
||||
|
||||
@Test(timeout = 120000)
|
||||
public void testDefaultAckQuorum() throws Exception {
|
||||
newBookie = bkutil.newBookie();
|
||||
int ensembleSize = numBookies + 1;
|
||||
int quorumSize = numBookies + 1;
|
||||
// ensure that the journal manager has to use all bookies,
|
||||
// so that a failure will fail the journal manager
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
|
||||
ensembleSize);
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
|
||||
quorumSize);
|
||||
// sets 2 secs
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
|
||||
2);
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"), nsi);
|
||||
bkjm.format(nsi);
|
||||
CountDownLatch sleepLatch = new CountDownLatch(1);
|
||||
sleepBookie(sleepLatch, newBookie);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
int numTransactions = 100;
|
||||
for (long i = 1; i <= numTransactions; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
try {
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, numTransactions);
|
||||
|
||||
List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
|
||||
bkjm.selectInputStreams(in, 1, true);
|
||||
try {
|
||||
assertEquals(numTransactions,
|
||||
FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
|
||||
} finally {
|
||||
in.get(0).close();
|
||||
}
|
||||
fail("Should throw exception as not enough non-faulty bookies available!");
|
||||
} catch (IOException ioe) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ack quorum feature supported by bookkeeper. Keep ack quorum bookie
|
||||
* alive and sleep all the other bookies. Now the client would wait for the
|
||||
* acknowledgement from the ack size bookies and after receiving the success
|
||||
* response will continue writing. Non ack client will hang long time to add
|
||||
* entries.
|
||||
*/
|
||||
@Test(timeout = 120000)
|
||||
public void testAckQuorum() throws Exception {
|
||||
// slow bookie
|
||||
newBookie = bkutil.newBookie();
|
||||
// make quorum size and ensemble size same to avoid the interleave writing
|
||||
// of the ledger entries
|
||||
int ensembleSize = numBookies + 1;
|
||||
int quorumSize = numBookies + 1;
|
||||
int ackSize = numBookies;
|
||||
// ensure that the journal manager has to use all bookies,
|
||||
// so that a failure will fail the journal manager
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
|
||||
ensembleSize);
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
|
||||
quorumSize);
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ACK_QUORUM_SIZE,
|
||||
ackSize);
|
||||
// sets 60 minutes
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ADD_ENTRY_TIMEOUT_SEC,
|
||||
3600);
|
||||
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-onebookiefailure"), nsi);
|
||||
bkjm.format(nsi);
|
||||
CountDownLatch sleepLatch = new CountDownLatch(1);
|
||||
sleepBookie(sleepLatch, newBookie);
|
||||
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
int numTransactions = 100;
|
||||
for (long i = 1; i <= numTransactions; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, numTransactions);
|
||||
|
||||
List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
|
||||
bkjm.selectInputStreams(in, 1, true);
|
||||
try {
|
||||
assertEquals(numTransactions,
|
||||
FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
|
||||
} finally {
|
||||
sleepLatch.countDown();
|
||||
in.get(0).close();
|
||||
bkjm.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sleep a bookie until I count down the latch
|
||||
*
|
||||
* @param latch
|
||||
* Latch to wait on
|
||||
* @param bookie
|
||||
* bookie server
|
||||
* @throws Exception
|
||||
*/
|
||||
private void sleepBookie(final CountDownLatch l, final BookieServer bookie)
|
||||
throws Exception {
|
||||
|
||||
Thread sleeper = new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
bookie.suspendProcessing();
|
||||
l.await(60, TimeUnit.SECONDS);
|
||||
bookie.resumeProcessing();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error suspending bookie", e);
|
||||
}
|
||||
}
|
||||
};
|
||||
sleeper.setName("BookieServerSleeper-" + bookie.getBookie().getId());
|
||||
sleeper.start();
|
||||
}
|
||||
|
||||
|
||||
private String startAndFinalizeLogSegment(BookKeeperJournalManager bkjm,
|
||||
int startTxid, int endTxid) throws IOException, KeeperException,
|
||||
InterruptedException {
|
||||
EditLogOutputStream out = bkjm.startLogSegment(startTxid,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long i = startTxid; i <= endTxid; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
// finalize the inprogress_1 log segment.
|
||||
bkjm.finalizeLogSegment(startTxid, endTxid);
|
||||
String zkpath1 = bkjm.finalizedLedgerZNode(startTxid, endTxid);
|
||||
assertNotNull(zkc.exists(zkpath1, false));
|
||||
assertNull(zkc.exists(bkjm.inprogressZNode(startTxid), false));
|
||||
return zkpath1;
|
||||
}
|
||||
}
|
|
@ -1,167 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.bookkeeper.proto.BookieServer;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestBookKeeperSpeculativeRead {
|
||||
private static final Log LOG = LogFactory
|
||||
.getLog(TestBookKeeperSpeculativeRead.class);
|
||||
|
||||
private ZooKeeper zkc;
|
||||
private static BKJMUtil bkutil;
|
||||
private static int numLocalBookies = 1;
|
||||
private static List<BookieServer> bks = new ArrayList<BookieServer>();
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBookkeeper() throws Exception {
|
||||
bkutil = new BKJMUtil(1);
|
||||
bkutil.start();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownBookkeeper() throws Exception {
|
||||
bkutil.teardown();
|
||||
for (BookieServer bk : bks) {
|
||||
bk.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
zkc = BKJMUtil.connectZooKeeper();
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws Exception {
|
||||
zkc.close();
|
||||
}
|
||||
|
||||
private NamespaceInfo newNSInfo() {
|
||||
Random r = new Random();
|
||||
return new NamespaceInfo(r.nextInt(), "testCluster", "TestBPID", -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test speculative read feature supported by bookkeeper. Keep one bookie
|
||||
* alive and sleep all the other bookies. Non spec client will hang for long
|
||||
* time to read the entries from the bookkeeper.
|
||||
*/
|
||||
@Test(timeout = 120000)
|
||||
public void testSpeculativeRead() throws Exception {
|
||||
// starting 9 more servers
|
||||
for (int i = 1; i < 10; i++) {
|
||||
bks.add(bkutil.newBookie());
|
||||
}
|
||||
NamespaceInfo nsi = newNSInfo();
|
||||
Configuration conf = new Configuration();
|
||||
int ensembleSize = numLocalBookies + 9;
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_ENSEMBLE_SIZE,
|
||||
ensembleSize);
|
||||
conf.setInt(BookKeeperJournalManager.BKJM_BOOKKEEPER_QUORUM_SIZE,
|
||||
ensembleSize);
|
||||
conf.setInt(
|
||||
BookKeeperJournalManager.BKJM_BOOKKEEPER_SPECULATIVE_READ_TIMEOUT_MS,
|
||||
100);
|
||||
// sets 60 minute
|
||||
conf.setInt(
|
||||
BookKeeperJournalManager.BKJM_BOOKKEEPER_READ_ENTRY_TIMEOUT_SEC, 3600);
|
||||
BookKeeperJournalManager bkjm = new BookKeeperJournalManager(conf,
|
||||
BKJMUtil.createJournalURI("/hdfsjournal-specread"), nsi);
|
||||
bkjm.format(nsi);
|
||||
|
||||
final long numTransactions = 1000;
|
||||
EditLogOutputStream out = bkjm.startLogSegment(1,
|
||||
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
|
||||
for (long i = 1; i <= numTransactions; i++) {
|
||||
FSEditLogOp op = FSEditLogTestUtil.getNoOpInstance();
|
||||
op.setTransactionId(i);
|
||||
out.write(op);
|
||||
}
|
||||
out.close();
|
||||
bkjm.finalizeLogSegment(1, numTransactions);
|
||||
|
||||
List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
|
||||
bkjm.selectInputStreams(in, 1, true);
|
||||
|
||||
// sleep 9 bk servers. Now only one server is running and responding to the
|
||||
// clients
|
||||
CountDownLatch sleepLatch = new CountDownLatch(1);
|
||||
for (final BookieServer bookie : bks) {
|
||||
sleepBookie(sleepLatch, bookie);
|
||||
}
|
||||
try {
|
||||
assertEquals(numTransactions,
|
||||
FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
|
||||
} finally {
|
||||
in.get(0).close();
|
||||
sleepLatch.countDown();
|
||||
bkjm.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sleep a bookie until I count down the latch
|
||||
*
|
||||
* @param latch
|
||||
* latch to wait on
|
||||
* @param bookie
|
||||
* bookie server
|
||||
* @throws Exception
|
||||
*/
|
||||
private void sleepBookie(final CountDownLatch latch, final BookieServer bookie)
|
||||
throws Exception {
|
||||
|
||||
Thread sleeper = new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
bookie.suspendProcessing();
|
||||
latch.await(2, TimeUnit.MINUTES);
|
||||
bookie.resumeProcessing();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error suspending bookie", e);
|
||||
}
|
||||
}
|
||||
};
|
||||
sleeper.setName("BookieServerSleeper-" + bookie.getBookie().getId());
|
||||
sleeper.start();
|
||||
}
|
||||
}
|
|
@ -1,170 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileFilter;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints.SlowCodec;
|
||||
import org.apache.hadoop.io.compress.CompressionCodecFactory;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
||||
public class TestBootstrapStandbyWithBKJM {
|
||||
private static BKJMUtil bkutil;
|
||||
protected MiniDFSCluster cluster;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBookkeeper() throws Exception {
|
||||
bkutil = new BKJMUtil(3);
|
||||
bkutil.start();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownBookkeeper() throws Exception {
|
||||
bkutil.teardown();
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
|
||||
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
|
||||
.createJournalURI("/bootstrapStandby").toString());
|
||||
BKJMUtil.addJournalManagerDefinition(conf);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
|
||||
conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY,
|
||||
SlowCodec.class.getCanonicalName());
|
||||
CompressionCodecFactory.setCodecClasses(conf,
|
||||
ImmutableList.<Class> of(SlowCodec.class));
|
||||
MiniDFSNNTopology topology = new MiniDFSNNTopology()
|
||||
.addNameservice(new MiniDFSNNTopology.NSConf("ns1").addNN(
|
||||
new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)).addNN(
|
||||
new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
|
||||
cluster = new MiniDFSCluster.Builder(conf).nnTopology(topology)
|
||||
.numDataNodes(1).manageNameDfsSharedDirs(false).build();
|
||||
cluster.waitActive();
|
||||
}
|
||||
|
||||
/**
|
||||
* While boostrapping, in_progress transaction entries should be skipped.
|
||||
* Bootstrap usage for BKJM : "-force", "-nonInteractive", "-skipSharedEditsCheck"
|
||||
*/
|
||||
@Test
|
||||
public void testBootstrapStandbyWithActiveNN() throws Exception {
|
||||
// make nn0 active
|
||||
cluster.transitionToActive(0);
|
||||
|
||||
// do ops and generate in-progress edit log data
|
||||
Configuration confNN1 = cluster.getConfiguration(1);
|
||||
DistributedFileSystem dfs = (DistributedFileSystem) HATestUtil
|
||||
.configureFailoverFs(cluster, confNN1);
|
||||
for (int i = 1; i <= 10; i++) {
|
||||
dfs.mkdirs(new Path("/test" + i));
|
||||
}
|
||||
dfs.close();
|
||||
|
||||
// shutdown nn1 and delete its edit log files
|
||||
cluster.shutdownNameNode(1);
|
||||
deleteEditLogIfExists(confNN1);
|
||||
cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_ENTER, true);
|
||||
cluster.getNameNodeRpc(0).saveNamespace(0, 0);
|
||||
cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_LEAVE, true);
|
||||
|
||||
// check without -skipSharedEditsCheck, Bootstrap should fail for BKJM
|
||||
// immediately after saveNamespace
|
||||
int rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive" },
|
||||
confNN1);
|
||||
Assert.assertEquals("Mismatches return code", 6, rc);
|
||||
|
||||
// check with -skipSharedEditsCheck
|
||||
rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive",
|
||||
"-skipSharedEditsCheck" }, confNN1);
|
||||
Assert.assertEquals("Mismatches return code", 0, rc);
|
||||
|
||||
// Checkpoint as fast as we can, in a tight loop.
|
||||
confNN1.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 1);
|
||||
cluster.restartNameNode(1);
|
||||
cluster.transitionToStandby(1);
|
||||
|
||||
NameNode nn0 = cluster.getNameNode(0);
|
||||
HATestUtil.waitForStandbyToCatchUp(nn0, cluster.getNameNode(1));
|
||||
long expectedCheckpointTxId = NameNodeAdapter.getNamesystem(nn0)
|
||||
.getFSImage().getMostRecentCheckpointTxId();
|
||||
HATestUtil.waitForCheckpoint(cluster, 1,
|
||||
ImmutableList.of((int) expectedCheckpointTxId));
|
||||
|
||||
// Should have copied over the namespace
|
||||
FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
|
||||
ImmutableList.of((int) expectedCheckpointTxId));
|
||||
FSImageTestUtil.assertNNFilesMatch(cluster);
|
||||
}
|
||||
|
||||
private void deleteEditLogIfExists(Configuration confNN1) {
|
||||
String editDirs = confNN1.get(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
|
||||
String[] listEditDirs = StringUtils.split(editDirs, ',');
|
||||
Assert.assertTrue("Wrong edit directory path!", listEditDirs.length > 0);
|
||||
|
||||
for (String dir : listEditDirs) {
|
||||
File curDir = new File(dir, "current");
|
||||
File[] listFiles = curDir.listFiles(new FileFilter() {
|
||||
@Override
|
||||
public boolean accept(File f) {
|
||||
if (!f.getName().startsWith("edits")) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
});
|
||||
if (listFiles != null && listFiles.length > 0) {
|
||||
for (File file : listFiles) {
|
||||
Assert.assertTrue("Failed to delete edit files!", file.delete());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,160 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.contrib.bkjournal;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.bookkeeper.util.LocalBookKeeper;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.server.NIOServerCnxnFactory;
|
||||
import org.apache.zookeeper.server.ZooKeeperServer;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Tests that read, update, clear api from CurrentInprogress
|
||||
*/
|
||||
public class TestCurrentInprogress {
|
||||
private static final Log LOG = LogFactory.getLog(TestCurrentInprogress.class);
|
||||
private static final String CURRENT_NODE_PATH = "/test";
|
||||
private static final String HOSTPORT = "127.0.0.1:2181";
|
||||
private static final int CONNECTION_TIMEOUT = 30000;
|
||||
private static NIOServerCnxnFactory serverFactory;
|
||||
private static ZooKeeperServer zks;
|
||||
private static ZooKeeper zkc;
|
||||
private static int ZooKeeperDefaultPort = 2181;
|
||||
private static File zkTmpDir;
|
||||
|
||||
private static ZooKeeper connectZooKeeper(String ensemble)
|
||||
throws IOException, KeeperException, InterruptedException {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
ZooKeeper zkc = new ZooKeeper(HOSTPORT, 3600, new Watcher() {
|
||||
public void process(WatchedEvent event) {
|
||||
if (event.getState() == Watcher.Event.KeeperState.SyncConnected) {
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
});
|
||||
if (!latch.await(10, TimeUnit.SECONDS)) {
|
||||
throw new IOException("Zookeeper took too long to connect");
|
||||
}
|
||||
return zkc;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupZooKeeper() throws Exception {
|
||||
LOG.info("Starting ZK server");
|
||||
zkTmpDir = File.createTempFile("zookeeper", "test");
|
||||
zkTmpDir.delete();
|
||||
zkTmpDir.mkdir();
|
||||
try {
|
||||
zks = new ZooKeeperServer(zkTmpDir, zkTmpDir, ZooKeeperDefaultPort);
|
||||
serverFactory = new NIOServerCnxnFactory();
|
||||
serverFactory.configure(new InetSocketAddress(ZooKeeperDefaultPort), 10);
|
||||
serverFactory.startup(zks);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception while instantiating ZooKeeper", e);
|
||||
}
|
||||
boolean b = LocalBookKeeper.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT);
|
||||
LOG.debug("ZooKeeper server up: " + b);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void shutDownServer() {
|
||||
if (null != zks) {
|
||||
zks.shutdown();
|
||||
}
|
||||
zkTmpDir.delete();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
zkc = connectZooKeeper(HOSTPORT);
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws Exception {
|
||||
if (null != zkc) {
|
||||
zkc.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that read should be able to read the data which updated with update
|
||||
* api
|
||||
*/
|
||||
@Test
|
||||
public void testReadShouldReturnTheZnodePathAfterUpdate() throws Exception {
|
||||
String data = "inprogressNode";
|
||||
CurrentInprogress ci = new CurrentInprogress(zkc, CURRENT_NODE_PATH);
|
||||
ci.init();
|
||||
ci.update(data);
|
||||
String inprogressNodePath = ci.read();
|
||||
assertEquals("Not returning inprogressZnode", "inprogressNode",
|
||||
inprogressNodePath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that read should return null if we clear the updated data in
|
||||
* CurrentInprogress node
|
||||
*/
|
||||
@Test
|
||||
public void testReadShouldReturnNullAfterClear() throws Exception {
|
||||
CurrentInprogress ci = new CurrentInprogress(zkc, CURRENT_NODE_PATH);
|
||||
ci.init();
|
||||
ci.update("myInprogressZnode");
|
||||
ci.read();
|
||||
ci.clear();
|
||||
String inprogressNodePath = ci.read();
|
||||
assertEquals("Expecting null to be return", null, inprogressNodePath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that update should throw IOE, if version number modifies between read
|
||||
* and update
|
||||
*/
|
||||
@Test(expected = IOException.class)
|
||||
public void testUpdateShouldFailWithIOEIfVersionNumberChangedAfterRead()
|
||||
throws Exception {
|
||||
CurrentInprogress ci = new CurrentInprogress(zkc, CURRENT_NODE_PATH);
|
||||
ci.init();
|
||||
ci.update("myInprogressZnode");
|
||||
assertEquals("Not returning myInprogressZnode", "myInprogressZnode", ci
|
||||
.read());
|
||||
// Updating data in-between to change the data to change the version number
|
||||
ci.update("YourInprogressZnode");
|
||||
ci.update("myInprogressZnode");
|
||||
}
|
||||
|
||||
}
|
|
@ -1,40 +0,0 @@
|
|||
/**
|
||||
* 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.hadoop.hdfs.server.namenode;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
|
||||
|
||||
/**
|
||||
* Utilities for testing edit logs
|
||||
*/
|
||||
public class FSEditLogTestUtil {
|
||||
private static OpInstanceCache cache = new OpInstanceCache();
|
||||
|
||||
public static FSEditLogOp getNoOpInstance() {
|
||||
return FSEditLogOp.LogSegmentOp.getInstance(cache,
|
||||
FSEditLogOpCodes.OP_END_LOG_SEGMENT);
|
||||
}
|
||||
|
||||
public static long countTransactionsInStream(EditLogInputStream in)
|
||||
throws IOException {
|
||||
FSEditLogLoader.EditLogValidation validation =
|
||||
FSEditLogLoader.scanEditLog(in, Long.MAX_VALUE);
|
||||
return (validation.getEndTxId() - in.getFirstTxId()) + 1;
|
||||
}
|
||||
}
|
|
@ -1,55 +0,0 @@
|
|||
#
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
#
|
||||
|
||||
#
|
||||
# Bookkeeper Journal Logging Configuration
|
||||
#
|
||||
|
||||
# Format is "<default threshold> (, <appender>)+
|
||||
|
||||
# DEFAULT: console appender only
|
||||
log4j.rootLogger=DEBUG, CONSOLE
|
||||
|
||||
# Example with rolling log file
|
||||
#log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE
|
||||
|
||||
# Example with rolling log file and tracing
|
||||
#log4j.rootLogger=TRACE, CONSOLE, ROLLINGFILE, TRACEFILE
|
||||
|
||||
#
|
||||
# Log INFO level and above messages to the console
|
||||
#
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
log4j.appender.CONSOLE.Threshold=INFO
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
|
||||
|
||||
#
|
||||
# Add ROLLINGFILE to rootLogger to get log file output
|
||||
# Log DEBUG level and above messages to a log file
|
||||
log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.ROLLINGFILE.Threshold=DEBUG
|
||||
log4j.appender.ROLLINGFILE.File=hdfs-namenode.log
|
||||
log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
|
||||
|
||||
# Max log file size of 10MB
|
||||
log4j.appender.ROLLINGFILE.MaxFileSize=10MB
|
|
@ -38,7 +38,6 @@ HDFS High Availability
|
|||
* [Securing access to ZooKeeper](#Securing_access_to_ZooKeeper)
|
||||
* [Verifying automatic failover](#Verifying_automatic_failover)
|
||||
* [Automatic Failover FAQ](#Automatic_Failover_FAQ)
|
||||
* [BookKeeper as a Shared storage (EXPERIMENTAL)](#BookKeeper_as_a_Shared_storage_EXPERIMENTAL)
|
||||
|
||||
Purpose
|
||||
-------
|
||||
|
@ -572,116 +571,3 @@ Automatic Failover FAQ
|
|||
using the same `hdfs haadmin` command. It will perform a coordinated
|
||||
failover.
|
||||
|
||||
BookKeeper as a Shared storage (EXPERIMENTAL)
|
||||
---------------------------------------------
|
||||
|
||||
One option for shared storage for the NameNode is BookKeeper. BookKeeper achieves high availability and strong durability guarantees by replicating edit log entries across multiple storage nodes. The edit log can be striped across the storage nodes for high performance. Fencing is supported in the protocol, i.e, BookKeeper will not allow two writers to write the single edit log.
|
||||
|
||||
The meta data for BookKeeper is stored in ZooKeeper. In current HA architecture, a Zookeeper cluster is required for ZKFC. The same cluster can be for BookKeeper metadata.
|
||||
|
||||
For more details on building a BookKeeper cluster, please refer to the [BookKeeper documentation](http://zookeeper.apache.org/bookkeeper/docs/trunk/bookkeeperConfig.html )
|
||||
|
||||
The BookKeeperJournalManager is an implementation of the HDFS JournalManager interface, which allows custom write ahead logging implementations to be plugged into the HDFS NameNode.
|
||||
|
||||
* **BookKeeper Journal Manager**
|
||||
|
||||
To use BookKeeperJournalManager, add the following to hdfs-site.xml.
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.shared.edits.dir</name>
|
||||
<value>bookkeeper://zk1:2181;zk2:2181;zk3:2181/hdfsjournal</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.edits.journal-plugin.bookkeeper</name>
|
||||
<value>org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager</value>
|
||||
</property>
|
||||
|
||||
The URI format for bookkeeper is `bookkeeper://[zkEnsemble]/[rootZnode] [zookkeeper ensemble]`
|
||||
is a list of semi-colon separated, zookeeper host:port
|
||||
pairs. In the example above there are 3 servers, in the ensemble,
|
||||
zk1, zk2 & zk3, each one listening on port 2181.
|
||||
|
||||
`[root znode]` is the path of the zookeeper znode, under which the edit log
|
||||
information will be stored.
|
||||
|
||||
The class specified for the journal-plugin must be available in the NameNode's
|
||||
classpath. We explain how to generate a jar file with the journal manager and
|
||||
its dependencies, and how to put it into the classpath below.
|
||||
|
||||
* **More configuration options**
|
||||
|
||||
* **dfs.namenode.bookkeeperjournal.output-buffer-size** -
|
||||
Number of bytes a bookkeeper journal stream will buffer before
|
||||
forcing a flush. Default is 1024.
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.bookkeeperjournal.output-buffer-size</name>
|
||||
<value>1024</value>
|
||||
</property>
|
||||
|
||||
* **dfs.namenode.bookkeeperjournal.ensemble-size** -
|
||||
Number of bookkeeper servers in edit log ensembles. This
|
||||
is the number of bookkeeper servers which need to be available
|
||||
for the edit log to be writable. Default is 3.
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.bookkeeperjournal.ensemble-size</name>
|
||||
<value>3</value>
|
||||
</property>
|
||||
|
||||
* **dfs.namenode.bookkeeperjournal.quorum-size** -
|
||||
Number of bookkeeper servers in the write quorum. This is the
|
||||
number of bookkeeper servers which must have acknowledged the
|
||||
write of an entry before it is considered written. Default is 2.
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.bookkeeperjournal.quorum-size</name>
|
||||
<value>2</value>
|
||||
</property>
|
||||
|
||||
* **dfs.namenode.bookkeeperjournal.digestPw** -
|
||||
Password to use when creating edit log segments.
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.bookkeeperjournal.digestPw</name>
|
||||
<value>myPassword</value>
|
||||
</property>
|
||||
|
||||
* **dfs.namenode.bookkeeperjournal.zk.session.timeout** -
|
||||
Session timeout for Zookeeper client from BookKeeper Journal Manager.
|
||||
Hadoop recommends that this value should be less than the ZKFC
|
||||
session timeout value. Default value is 3000.
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.bookkeeperjournal.zk.session.timeout</name>
|
||||
<value>3000</value>
|
||||
</property>
|
||||
|
||||
* **Building BookKeeper Journal Manager plugin jar**
|
||||
|
||||
To generate the distribution packages for BK journal, do the following.
|
||||
|
||||
$ mvn clean package -Pdist
|
||||
|
||||
This will generate a jar with the BookKeeperJournalManager,
|
||||
hadoop-hdfs/src/contrib/bkjournal/target/hadoop-hdfs-bkjournal-*VERSION*.jar
|
||||
|
||||
Note that the -Pdist part of the build command is important, this would
|
||||
copy the dependent bookkeeper-server jar under
|
||||
hadoop-hdfs/src/contrib/bkjournal/target/lib.
|
||||
|
||||
* **Putting the BookKeeperJournalManager in the NameNode classpath**
|
||||
|
||||
To run a HDFS namenode using BookKeeper as a backend, copy the bkjournal and
|
||||
bookkeeper-server jar, mentioned above, into the lib directory of hdfs. In the
|
||||
standard distribution of HDFS, this is at $HADOOP\_HDFS\_HOME/share/hadoop/hdfs/lib/
|
||||
|
||||
cp hadoop-hdfs/src/contrib/bkjournal/target/hadoop-hdfs-bkjournal-*VERSION*.jar $HADOOP\_HDFS\_HOME/share/hadoop/hdfs/lib/
|
||||
|
||||
* **Current limitations**
|
||||
|
||||
1) Security in BookKeeper. BookKeeper does not support SASL nor SSL for
|
||||
connections between the NameNode and BookKeeper storage nodes.
|
||||
|
||||
|
||||
|
|
|
@ -35,7 +35,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|||
<module>hadoop-hdfs-client</module>
|
||||
<module>hadoop-hdfs-native-client</module>
|
||||
<module>hadoop-hdfs-httpfs</module>
|
||||
<module>hadoop-hdfs/src/contrib/bkjournal</module>
|
||||
<module>hadoop-hdfs-nfs</module>
|
||||
</modules>
|
||||
|
||||
|
|
|
@ -929,12 +929,6 @@
|
|||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.bookkeeper</groupId>
|
||||
<artifactId>bookkeeper-server</artifactId>
|
||||
<version>4.2.3</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hsqldb</groupId>
|
||||
<artifactId>hsqldb</artifactId>
|
||||
|
|
Loading…
Reference in New Issue