Skip to content
This repository was archived by the owner on Oct 16, 2024. It is now read-only.

Commit

Permalink
ISSUE #209: Support rename log
Browse files Browse the repository at this point in the history
Descriptions of the changes in this PR:

- add rename operation in `Namespace`
- add rename operation in `LogStreamMetadataStore`
- implement the rename operation use zookeeper `multi` operation

Author: Sijie Guo <sijie@apache.org>
Author: Shoukun Huai <shoukunhuai@gmail.com>
Author: Arvin <arvindevel@gmail.com>

Reviewers: Jia Zhai <None>

This closes #210 from sijie/4_support_rename_pr, closes #209
  • Loading branch information
sijie authored and jiazhai committed Oct 23, 2017
1 parent 298a23b commit b670382
Show file tree
Hide file tree
Showing 5 changed files with 601 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.net.URI;
import java.util.Iterator;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.bookkeeper.feature.FeatureProvider;
Expand All @@ -33,6 +34,7 @@
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.namespace.Namespace;
import org.apache.distributedlog.callback.NamespaceListener;
import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.common.util.PermitLimiter;
import org.apache.distributedlog.common.util.SchedulerUtils;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
Expand Down Expand Up @@ -193,6 +195,28 @@ public DistributedLogManager openLog(String logName,
dynamicLogConf);
}

@Override
public CompletableFuture<Void> renameLog(String oldName, String newName) {
try {
checkState();
final String oldLogName = validateAndNormalizeName(oldName);
final String newLogName = validateAndNormalizeName(newName);

return driver.getLogMetadataStore().getLogLocation(oldName)
.thenCompose(uriOptional -> {
if (uriOptional.isPresent()) {
return driver.getLogStreamMetadataStore(WRITER)
.renameLog(uriOptional.get(), oldLogName, newLogName);
} else {
return FutureUtils.exception(
new LogNotFoundException("Log " + oldLogName + " isn't found."));
}
});
} catch (IOException ioe) {
return FutureUtils.exception(ioe);
}
}

@Override
public boolean logExists(String logName)
throws IOException, IllegalArgumentException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.io.IOException;
import java.util.Iterator;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public;
import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving;
import org.apache.bookkeeper.stats.StatsLogger;
Expand Down Expand Up @@ -106,6 +107,19 @@ void createLog(String logName)
void deleteLog(String logName)
throws InvalidStreamNameException, LogNotFoundException, IOException;

/**
* Rename a log from <i>oldName</i> to <i>newName</i>.
*
* @param oldName old log name
* @param newName new log name
* @return a future represents the rename result.
* @throws InvalidStreamNameException if log name is invalid
* @throws LogNotFoundException if old log doesn't exist
* @throws org.apache.distributedlog.exceptions.LogExistsException if the new log exists
* @throws IOException when encountered issues with backend.
*/
CompletableFuture<Void> renameLog(String oldName, String newName);

/**
* Open a log named <i>logName</i>.
* A distributedlog manager is returned to access log <i>logName</i>.
Expand Down
Loading

0 comments on commit b670382

Please sign in to comment.