This commit fixes test setup in SnapshotUtils. Because of the incorrect test setup
the execution is providing incorrect results. For example, assume the following path
/cassandra-test/data/ks/tbl/snapshots/test-snapshot
The test was configuring data directories as ["/cassandra-test/data"], but in a real
execution data directories is provided as ["/cassandra-test"]. This is causing the
endpoint to return incorrect values in the JSON payload.
Additionally, the response was providing the port for Cassandra and not the Sidecar
port.
import java.util.List;
import java.util.Objects;
+import com.google.common.annotations.Beta;
+
import com.fasterxml.jackson.annotation.JsonProperty;
/**
- * A class representing a response for the {@link ListSnapshotFilesRequest}
+ * A class representing a response for the {@link ListSnapshotFilesRequest}.
+ * This class is expected to evolve and has been mark with the {@link Beta} annotation.
*/
+@Beta
public class ListSnapshotFilesResponse
{
private final List<FileInfo> snapshotFilesInfo;
version=1.0-SNAPSHOT
junitVersion=5.4.2
kubernetesClientVersion=9.0.0
-cassandra40Version=4.0.4
+cassandra40Version=4.0.5
vertxVersion=4.2.1
<Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
</Match>
+ <!-- Ignore DMI_HARDCODED_ABSOLUTE_FILENAME for testing SnapshotDirectory.of with strings that are paths -->
+ <Match>
+ <Class name="org.apache.cassandra.sidecar.snapshots.SnapshotDirectoryTest" />
+ <Bug pattern="DMI_HARDCODED_ABSOLUTE_FILENAME" />
+ </Match>
+
</FindBugsFilter>
\ No newline at end of file
import java.io.FileNotFoundException;
import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
import java.util.List;
-import org.apache.commons.lang3.tuple.Pair;
+import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.inject.Inject;
import io.netty.handler.codec.http.HttpResponseStatus;
-import io.vertx.core.file.FileProps;
import io.vertx.core.http.HttpServerRequest;
import io.vertx.core.net.SocketAddress;
import io.vertx.ext.web.RoutingContext;
import io.vertx.ext.web.handler.HttpException;
-import org.apache.cassandra.sidecar.cluster.InstancesConfig;
-import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.Configuration;
import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesRequest;
import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesResponse;
+import org.apache.cassandra.sidecar.snapshots.SnapshotDirectory;
import org.apache.cassandra.sidecar.snapshots.SnapshotPathBuilder;
/**
* For example:
*
* <p>
- * /api/v1/snapshots/testSnapshot lists all SSTable component files for all the
- * "testSnapshot" snapshots
- * <p>
- * /api/v1/snapshots/testSnapshot?includeSecondaryIndexFiles=true lists all SSTable component files including
- * secondary index files for all the "testSnapshot"
- * snapshots
- * <p>
- * /api/v1/keyspace/ks/table/tbl/snapshots/testSnapshot lists all SSTable component files for the
- * "testSnapshot" snapshot for the "ks" keyspace
- * and the "tbl" table
+ * /api/v1/keyspace/ks/table/tbl/snapshots/testSnapshot
+ * lists all SSTable component files for the "testSnapshot" snapshot for the "ks" keyspace and the "tbl" table
* <p>
* /api/v1/keyspace/ks/table/tbl/snapshots/testSnapshot?includeSecondaryIndexFiles=true
- * lists all SSTable component files including
- * secondary index files for the "testSnapshot"
- * snapshot for the "ks" keyspace and the "tbl"
- * table
+ * lists all SSTable component files including secondary index files for the "testSnapshot" snapshot for the "ks"
+ * keyspace and the "tbl" table
*/
public class ListSnapshotFilesHandler extends AbstractHandler
{
private static final Logger logger = LoggerFactory.getLogger(ListSnapshotFilesHandler.class);
private static final String INCLUDE_SECONDARY_INDEX_FILES = "includeSecondaryIndexFiles";
- private static final int DATA_DIR_INDEX = 0;
- private static final int TABLE_NAME_SUBPATH_INDEX = 1;
- private static final int FILE_NAME_SUBPATH_INDEX = 4;
private final SnapshotPathBuilder builder;
+ private final Configuration configuration;
@Inject
- public ListSnapshotFilesHandler(SnapshotPathBuilder builder, InstancesConfig instancesConfig)
+ public ListSnapshotFilesHandler(SnapshotPathBuilder builder, Configuration configuration)
{
- super(instancesConfig);
+ super(configuration.getInstancesConfig());
this.builder = builder;
+ this.configuration = configuration;
}
@Override
boolean secondaryIndexFiles = requestParams.includeSecondaryIndexFiles();
builder.build(host, requestParams)
- .compose(directory -> builder.listSnapshotDirectory(directory, secondaryIndexFiles))
- .onSuccess(fileList ->
- {
- if (fileList.isEmpty())
- {
- String payload = "Snapshot '" + requestParams.getSnapshotName() + "' not found";
- context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(), payload));
- }
- else
- {
- logger.debug("ListSnapshotFilesHandler handled {} for {}. Instance: {}",
- requestParams, remoteAddress, host);
- context.json(buildResponse(host, requestParams, fileList));
- }
- })
- .onFailure(cause ->
- {
- logger.error("ListSnapshotFilesHandler failed for request: {} from: {}. Instance: {}",
- requestParams, remoteAddress, host);
- if (cause instanceof FileNotFoundException ||
- cause instanceof NoSuchFileException)
- {
- context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(),
- cause.getMessage()));
- }
- else
- {
- context.fail(new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
- "Invalid request for " + requestParams));
- }
- });
+ .onSuccess(snapshotDirectory ->
+ builder.listSnapshotDirectory(snapshotDirectory, secondaryIndexFiles)
+ .onSuccess(fileList ->
+ {
+ if (fileList.isEmpty())
+ {
+ String payload = "Snapshot '" + requestParams.getSnapshotName() +
+ "' not found";
+ context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(), payload));
+ }
+ else
+ {
+ logger.debug("ListSnapshotFilesHandler handled {} for {}. Instance: {}",
+ requestParams, remoteAddress, host);
+ context.json(buildResponse(host, snapshotDirectory, fileList));
+ }
+ })
+ .onFailure(cause -> processFailure(cause, context, requestParams, remoteAddress, host))
+ )
+ .onFailure(cause -> processFailure(cause, context, requestParams, remoteAddress, host));
+ }
+
+ private void processFailure(Throwable cause, RoutingContext context, ListSnapshotFilesRequest requestParams,
+ SocketAddress remoteAddress, String host)
+ {
+ logger.error("ListSnapshotFilesHandler failed for request: {} from: {}. Instance: {}",
+ requestParams, remoteAddress, host);
+ if (cause instanceof FileNotFoundException ||
+ cause instanceof NoSuchFileException)
+ {
+ context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(),
+ cause.getMessage()));
+ }
+ else
+ {
+ context.fail(new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
+ "Invalid request for " + requestParams));
+ }
}
private ListSnapshotFilesResponse buildResponse(String host,
- ListSnapshotFilesRequest request,
- List<Pair<String, FileProps>> fileList)
+ String snapshotDirectory,
+ List<SnapshotPathBuilder.SnapshotFile> fileList)
{
- InstanceMetadata instanceMetadata = instancesConfig.instanceFromHost(host);
- int sidecarPort = instanceMetadata.port();
- Path dataDirPath = Paths.get(instanceMetadata.dataDirs().get(DATA_DIR_INDEX));
ListSnapshotFilesResponse response = new ListSnapshotFilesResponse();
- String snapshotName = request.getSnapshotName();
+ int sidecarPort = configuration.getPort();
+ SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+ int dataDirectoryIndex = dataDirectoryIndex(host, directory.dataDirectory);
+ int offset = snapshotDirectory.length() + 1;
- for (Pair<String, FileProps> file : fileList)
+ for (SnapshotPathBuilder.SnapshotFile snapshotFile : fileList)
{
- Path pathFromDataDir = dataDirPath.relativize(Paths.get(file.getLeft()));
-
- String keyspace = request.getKeyspace();
- // table name might include a dash (-) with the table UUID so we always use it as part of the response
- String tableName = pathFromDataDir.getName(TABLE_NAME_SUBPATH_INDEX).toString();
- String fileName = pathFromDataDir.getName(FILE_NAME_SUBPATH_INDEX).toString();
-
- response.addSnapshotFile(new ListSnapshotFilesResponse.FileInfo(file.getRight().size(),
- host,
- sidecarPort,
- DATA_DIR_INDEX,
- snapshotName,
- keyspace,
- tableName,
- fileName));
+ int fileNameIndex = snapshotFile.path.indexOf(snapshotDirectory) + offset;
+ Preconditions.checkArgument(fileNameIndex < snapshotFile.path.length(),
+ "Invalid snapshot file '" + snapshotFile.path + "'");
+ response.addSnapshotFile(
+ new ListSnapshotFilesResponse.FileInfo(snapshotFile.size,
+ host,
+ sidecarPort,
+ dataDirectoryIndex,
+ directory.snapshotName,
+ directory.keyspace,
+ directory.tableName,
+ snapshotFile.path.substring(fileNameIndex)));
}
return response;
}
+ private int dataDirectoryIndex(String host, String dataDirectory)
+ {
+ List<String> dataDirs = instancesConfig.instanceFromHost(host).dataDirs();
+ for (int index = 0; index < dataDirs.size(); index++)
+ {
+ if (dataDirectory.startsWith(dataDirs.get(index)))
+ {
+ return index;
+ }
+ }
+ return -1;
+ }
+
private ListSnapshotFilesRequest extractParamsOrThrow(final RoutingContext context)
{
boolean includeSecondaryIndexFiles =
--- /dev/null
+package org.apache.cassandra.sidecar.snapshots;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import com.google.common.base.Preconditions;
+
+import static org.apache.cassandra.sidecar.snapshots.SnapshotPathBuilder.SNAPSHOTS_DIR_NAME;
+
+/**
+ * An object that encapsulates the parts of a snapshot directory
+ */
+public class SnapshotDirectory
+{
+ public final String dataDirectory;
+ public final String keyspace;
+ public final String tableName;
+ public final String snapshotName;
+
+ SnapshotDirectory(String dataDirectory, String keyspace, String tableName, String snapshotName)
+ {
+ this.dataDirectory = dataDirectory;
+ this.keyspace = keyspace;
+ this.tableName = tableName;
+ this.snapshotName = snapshotName;
+ }
+
+ /**
+ * Parses a snapshot directory string into a {@link SnapshotDirectory} object. The snapshot directory
+ * has the following structure {@code /<data_dir>/<ks>/<table>/snapshots/<snapshot_name>}.
+ *
+ * @param snapshotDirectory the absolute path to the snapshot directory
+ * @return the {@link SnapshotDirectory} object representing the provided {@code snapshotDirectory}
+ */
+ public static SnapshotDirectory of(String snapshotDirectory)
+ {
+ Path snapshotDirectoryPath = Paths.get(snapshotDirectory);
+ int nameCount = snapshotDirectoryPath.getNameCount();
+ Preconditions.checkArgument(nameCount >= 5, "Invalid snapshotDirectory. " +
+ "Expected at least 5 parts but found " + nameCount);
+ String snapshotName = snapshotDirectoryPath.getName(nameCount - 1).toString();
+ String snapshotDirName = snapshotDirectoryPath.getName(nameCount - 2).toString();
+ String tableName = snapshotDirectoryPath.getName(nameCount - 3).toString();
+ String keyspace = snapshotDirectoryPath.getName(nameCount - 4).toString();
+ String dataDirectory = File.separator + snapshotDirectoryPath.subpath(0, nameCount - 4);
+
+ Preconditions.checkArgument(SNAPSHOTS_DIR_NAME.equalsIgnoreCase(snapshotDirName),
+ "Invalid snapshotDirectory. The expected directory structure is " +
+ "'/<data_dir>/<ks>/<table>/snapshots/<snapshot_name>'");
+
+ return new SnapshotDirectory(dataDirectory, keyspace, tableName, snapshotName);
+ }
+}
{
private static final Logger logger = LoggerFactory.getLogger(SnapshotPathBuilder.class);
private static final String DATA_SUB_DIR = "/data";
- public static final int SNAPSHOTS_MAX_DEPTH = 4;
+ public static final int SNAPSHOTS_MAX_DEPTH = 5;
public static final String SNAPSHOTS_DIR_NAME = "snapshots";
protected final Vertx vertx;
protected final FileSystem fs;
* @param includeSecondaryIndexFiles whether to include secondary index files
* @return a future with a list of files inside the snapshot directory
*/
- public Future<List<Pair<String, FileProps>>> listSnapshotDirectory(String snapshotDirectory,
- boolean includeSecondaryIndexFiles)
+ public Future<List<SnapshotFile>> listSnapshotDirectory(String snapshotDirectory,
+ boolean includeSecondaryIndexFiles)
{
- Promise<List<Pair<String, FileProps>>> promise = Promise.promise();
+ Promise<List<SnapshotFile>> promise = Promise.promise();
// List the snapshot directory
fs.readDir(snapshotDirectory)
{
// Create a pair of path/fileProps for every regular file
- List<Pair<String, FileProps>> snapshotList =
+ List<SnapshotFile> snapshotList =
IntStream.range(0, list.size())
.filter(i -> ar.<FileProps>resultAt(i).isRegularFile())
- .mapToObj(i -> Pair.of(list.get(i), ar.<FileProps>resultAt(i)))
+ .mapToObj(i ->
+ {
+ long size = ar.<FileProps>resultAt(i).size();
+ return new SnapshotFile(list.get(i),
+ size);
+ })
.collect(Collectors.toList());
.onSuccess(idx ->
{
//noinspection unchecked
- List<Pair<String, FileProps>> idxPropList =
+ List<SnapshotFile> idxPropList =
idx.list()
.stream()
- .flatMap(l -> ((List<Pair<String, FileProps>>) l).stream())
+ .flatMap(l -> ((List<SnapshotFile>) l).stream())
.collect(Collectors.toList());
// aggregate the results and return the full list
return vertx.executeBlocking(promise ->
{
-
// a filter to keep directories ending in "/snapshots/<snapshotName>"
BiPredicate<Path, BasicFileAttributes> filter = (path, basicFileAttributes) ->
{
});
return promise.future();
}
+
+ /**
+ * Class representing a snapshot component file
+ */
+ public static class SnapshotFile
+ {
+ public final String path;
+ public final long size;
+
+ SnapshotFile(String path, long size)
+ {
+ this.path = path;
+ this.size = size;
+ }
+ }
}
@Provides
@Singleton
- public Configuration configuration()
+ public Configuration configuration(InstancesConfig instancesConfig)
{
- return abstractConfig();
+ return abstractConfig(instancesConfig);
}
- protected Configuration abstractConfig()
+ protected Configuration abstractConfig(InstancesConfig instancesConfig)
{
return new Configuration.Builder()
- .setInstancesConfig(getInstancesConfig())
+ .setInstancesConfig(instancesConfig)
.setHost("127.0.0.1")
.setPort(6475)
.setHealthCheckFrequency(1000)
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.cassandra.sidecar.cluster.InstancesConfig;
+
/**
* Changes to the TestModule to define SSL dependencies
*/
@Override
- public Configuration abstractConfig()
+ public Configuration abstractConfig(InstancesConfig instancesConfig)
{
final String keyStorePath = TestSslModule.class.getClassLoader().getResource("certs/test.p12").getPath();
final String keyStorePassword = "password";
}
return new Configuration.Builder()
- .setInstancesConfig(getInstancesConfig())
+ .setInstancesConfig(instancesConfig)
.setHost("127.0.0.1")
.setPort(6475)
.setHealthCheckFrequency(1000)
import java.io.File;
import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import com.google.inject.AbstractModule;
import com.google.inject.Guice;
import com.google.inject.Injector;
+import com.google.inject.Provides;
+import com.google.inject.Singleton;
import com.google.inject.util.Modules;
import io.vertx.core.Vertx;
import io.vertx.core.http.HttpServer;
ListSnapshotFilesResponse.FileInfo fileInfoExpected =
new ListSnapshotFilesResponse.FileInfo(11,
"localhost",
- 9043,
+ 6475,
0,
"snapshot1",
"keyspace1",
ListSnapshotFilesResponse.FileInfo fileInfoNotExpected =
new ListSnapshotFilesResponse.FileInfo(11,
"localhost",
- 9043,
+ 6475,
0,
"snapshot1",
"keyspace1",
})));
}
+ @Test
+ public void testRouteSucceedsIncludeSecondaryIndexes(VertxTestContext context)
+ {
+ WebClient client = WebClient.create(vertx);
+ String testRoute = "/api/v1/keyspace/keyspace1/table/table1-1234" +
+ "/snapshots/snapshot1?includeSecondaryIndexFiles=true";
+ List<ListSnapshotFilesResponse.FileInfo> fileInfoExpected = Arrays.asList(
+ new ListSnapshotFilesResponse.FileInfo(11,
+ "localhost",
+ 6475,
+ 0,
+ "snapshot1",
+ "keyspace1",
+ "table1-1234",
+ "1.db"),
+ new ListSnapshotFilesResponse.FileInfo(0,
+ "localhost",
+ 6475,
+ 0,
+ "snapshot1",
+ "keyspace1",
+ "table1-1234",
+ ".index/secondary.db")
+ );
+ ListSnapshotFilesResponse.FileInfo fileInfoNotExpected =
+ new ListSnapshotFilesResponse.FileInfo(11,
+ "localhost",
+ 6475,
+ 0,
+ "snapshot1",
+ "keyspace1",
+ "table1-1234",
+ "2.db");
+
+ client.get(config.getPort(), "localhost", testRoute)
+ .send(context.succeeding(response -> context.verify(() ->
+ {
+ assertThat(response.statusCode()).isEqualTo(OK.code());
+ ListSnapshotFilesResponse resp = response.bodyAsJson(ListSnapshotFilesResponse.class);
+ assertThat(resp.getSnapshotFilesInfo()).containsAll(fileInfoExpected);
+ assertThat(resp.getSnapshotFilesInfo()).doesNotContain(fileInfoNotExpected);
+ context.completeNow();
+ })));
+ }
+
@Test
public void testRouteInvalidSnapshot(VertxTestContext context)
{
class ListSnapshotTestModule extends AbstractModule
{
- @Override
- protected void configure()
+ @Provides
+ @Singleton
+ public InstancesConfig getInstancesConfig() throws IOException
{
- try
- {
- bind(InstancesConfig.class).toInstance(mockInstancesConfig(temporaryFolder.getCanonicalPath()));
- }
- catch (IOException e)
- {
- throw new RuntimeException(e);
- }
+ return mockInstancesConfig(temporaryFolder.getCanonicalPath());
}
}
}
--- /dev/null
+package org.apache.cassandra.sidecar.snapshots;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException;
+
+class SnapshotDirectoryTest
+{
+
+ @ParameterizedTest
+ @ValueSource(strings = { "not-valid", "/two-levels/not-valid", "three/levels/not-valid", "four/levels/not/valid" })
+ void failsOnInvalidLengthDirectory()
+ {
+ assertThatIllegalArgumentException()
+ .isThrownBy(() -> SnapshotDirectory.of("not-valid"))
+ .withMessageContaining("Invalid snapshotDirectory. Expected at least 5 parts but found");
+ }
+
+ @Test
+ void failsOnInvalidDirectory()
+ {
+ assertThatIllegalArgumentException()
+ .isThrownBy(() -> SnapshotDirectory.of("/cassandra/data/ks1/tbl2/sneaky/test-snapshot"))
+ .withMessage("Invalid snapshotDirectory. The expected directory structure is " +
+ "'/<data_dir>/<ks>/<table>/snapshots/<snapshot_name>'");
+ }
+
+ @Test
+ void testValidDirectory1()
+ {
+ String snapshotDirectory = "/cassandra/data/ks1/tbl2/snapshots/test-snapshot";
+ SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+ assertThat(directory.dataDirectory).isEqualTo("/cassandra/data");
+ assertThat(directory.keyspace).isEqualTo("ks1");
+ assertThat(directory.tableName).isEqualTo("tbl2");
+ assertThat(directory.snapshotName).isEqualTo("test-snapshot");
+ }
+
+ @Test
+ void testValidDirectory2()
+ {
+ String snapshotDirectory = "/cassandra/data/ks1/tbl2/SNAPSHOTS/test-snapshot";
+ SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+ assertThat(directory.dataDirectory).isEqualTo("/cassandra/data");
+ assertThat(directory.keyspace).isEqualTo("ks1");
+ assertThat(directory.tableName).isEqualTo("tbl2");
+ assertThat(directory.snapshotName).isEqualTo("test-snapshot");
+ }
+
+ @Test
+ void testValidDirectory3()
+ {
+ String snapshotDirectory = "/datadir/inventory/shipping/snapshots/2022-07-23";
+ SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+ assertThat(directory.dataDirectory).isEqualTo("/datadir");
+ assertThat(directory.keyspace).isEqualTo("inventory");
+ assertThat(directory.tableName).isEqualTo("shipping");
+ assertThat(directory.snapshotName).isEqualTo("2022-07-23");
+ }
+
+ @Test
+ void testValidDirectory4()
+ {
+ String snapshotDirectory = "/cassandra/disk1/data/inventory/shipping/snapshots/2022-07-23/";
+ SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+ assertThat(directory.dataDirectory).isEqualTo("/cassandra/disk1/data");
+ assertThat(directory.keyspace).isEqualTo("inventory");
+ assertThat(directory.tableName).isEqualTo("shipping");
+ assertThat(directory.snapshotName).isEqualTo("2022-07-23");
+ }
+}
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
-import org.apache.commons.lang3.tuple.Pair;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import io.vertx.core.CompositeFuture;
import io.vertx.core.Future;
import io.vertx.core.Vertx;
-import io.vertx.core.file.FileProps;
import io.vertx.junit5.VertxExtension;
import io.vertx.junit5.VertxTestContext;
import org.apache.cassandra.sidecar.cluster.InstancesConfig;
//noinspection unchecked
List<String> snapshotFiles = ar.list()
.stream()
- .flatMap(l -> ((List<Pair<String, FileProps>>) l).stream())
- .map(Pair::getLeft)
+ .flatMap(l -> ((List<SnapshotPathBuilder.SnapshotFile>) l).stream())
+ .map(snapshotFile -> snapshotFile.path)
.sorted()
.collect(Collectors.toList());
InstanceMetadataImpl localhost = new InstanceMetadataImpl(1,
"localhost",
9043,
- Collections.singletonList(rootPath + "/d1/data"),
+ Collections.singletonList(rootPath + "/d1"),
null,
versionProvider,
1000);
InstanceMetadataImpl localhost2 = new InstanceMetadataImpl(2,
"localhost2",
9043,
- Collections.singletonList(rootPath + "/d2/data"),
+ Collections.singletonList(rootPath + "/d2"),
null,
versionProvider,
1000);