bruno-roustant commented on a change in pull request #39:
URL: https://github.com/apache/solr/pull/39#discussion_r617397694



##########
File path: solr/solr-ref-guide/src/making-and-restoring-backups.adoc
##########
@@ -223,23 +223,190 @@ Request ID to track this action which will be processed 
asynchronously.
 
 == Backup/Restore Storage Repositories
 
-Solr provides interfaces to plug different storage systems for backing up and 
restoring. For example, you can have a Solr cluster running on a local 
filesystem like EXT3 but you can backup the indexes to a HDFS filesystem or 
vice versa.
+Solr provides a repository abstraction to allow users to backup and restore 
their data to a variety of different storage systems.
+For example, a Solr cluster running on a local filesystem (e.g. EXT3) can 
store backup data on the same disk, on a remote network-mounted drive, in HDFS, 
or even in some popular "cloud storage" providers, depending on the 
'repository' implementation chosen.
+Solr offers three different repository implementations out of the box 
(`LocalFileSystemRepository`, `HdfsBackupRepository`, and 
`GCSBackupRepository`), and allows users to create plugins for their own 
storage systems as needed.
 
-The repository interfaces needs to be configured in the `solr.xml` file. While 
running backup/restore commands we can specify the repository to be used.
+Users can define any number of repositories in their `solr.xml` file.
+The backup and restore APIs described above allow users to select which of 
these definitions they want to use at runtime via the `repository` parameter.
+When no `repository` parameter is specified, the local filesystem repository 
is used as a default.
 
-If no repository is configured then the local filesystem repository will be 
used automatically.
+Repositories are defined by a `<repository>` tag nested under a `<backup>` 
parent tag.
+All `<repository>` tags must have a `name` attribute (defines the identifier 
that users can reference later to select this repository) and a `class` 
attribute (containing the full Java classname that implements the repository).
+They may also have a boolean `default` attribute, which may be `true` on at 
most one repository definition.
+Any children under the `<repository>` tag are passed as additional 
configuration to the repository, allowing repositories to read their own 
implementation-specific configuration.
 
-Example `solr.xml` section to configure a repository like 
<<running-solr-on-hdfs.adoc#,HDFS>>:
+Information on each of the repository implementations provided with Solr is 
provided below.
+
+=== LocalFileSystemRepository
+
+LocalFileSystemRepository stores and retrieves backup files anywhere on the 
accessible filesystem.
+Files can be stored on "local" disk, or on network-mounted drives that appear 
local to the filesystem.
+
+WARNING: SolrCloud administrators looking to use LocalFileSystemRepository in 
tandem with network drives should be careful to make the drive available at the 
same location on each Solr node.
+Strictly speaking, the mount only needs to be present on the node doing the 
backup (or restore), and on the node currently serving as the "Overseer".
+However since the "overseer" role often moves from node to node in a cluster, 
it is generally recommended that backup drives be added to all nodes uniformly.
+
+A LocalFileSystemRepository instance is used as a default by any backup and 
restore commands that don't explicitly provide a `repository` parameter or have 
a default specified in `solr.xml`.
+
+LocalFileSystemRepository accepts the following configuration options:
+
+`location`::
+A valid file path (accessible to Solr locally) to use for backup storage and 
retrieval.  Used as a fallback when user's don't provide a `location` parameter 
in their Backup or Restore API commands
+
+An example configuration using this property can be found below.
+
+[source,xml]
+----
+<backup>
+  <repository name="local_repo" 
class="org.apache.solr.core.backup.repository.LocalFileSytemRepository">
+    <str name="location">/solr/backup_data</str>
+  </repository>
+</backup>
+----
+
+
+=== HdfsBackupRepository
+
+Stores and retrieves backup files from HDFS directories.
+
+WARNING: HdfsBackupRepository is deprecated and may be removed or relocated in 
a subsequent version of Solr.
+
+HdfsBackupRepository accepts the following configuration options:
+
+`solr.hdfs.buffer.size`::
+The size, in bytes, of the buffer used to transfer data to and from HDFS.
+Defaults to 4096 (4KB).
+Better throughput is often attainable with a larger buffer, where memory 
allows.
+
+`solr.hdfs.home`::
+Required.
+A HDFS URI in the format `hdfs://<host>:<port>/<hdfsBaseFilePath>` that points 
Solr to the HDFS cluster to store (or retrieve) backup files on.
+
+`solr.hdfs.permissions.umask-mode`::
+A permission umask used when creating files in HDFS.
+
+`location`::
+A valid directory path on the HDFS cluster to use for backup storage and 
retrieval.  Used as a fallback when users don't provide a `location` parameter 
in their Backup or Restore API commands
+
+An example configuration using these properties can be found below:
 
 [source,xml]
 ----
 <backup>
   <repository name="hdfs" 
class="org.apache.solr.core.backup.repository.HdfsBackupRepository" 
default="false">
-    <str name="location">${solr.hdfs.default.backup.path}</str>
-    <str name="solr.hdfs.home">${solr.hdfs.home:}</str>
-    <str name="solr.hdfs.confdir">${solr.hdfs.confdir:}</str>
+    <str 
name="solr.hdfs.home">hdfs://some_hdfs_host:1234/solr/backup/data</str>
+    <int name="solr.hdfs.buffer.size">8192</int>
+    <str name="solr.hdfs.permissions.umask-mode">0022</str>
+    <str name="location">/default/hdfs/backup/location</str>
   </repository>
 </backup>
 ----
 
-Better throughput might be achieved by increasing buffer size with `<int 
name="solr.hdfs.buffer.size">262144</int>`. Buffer size is specified in bytes, 
by default it's 4096 bytes (4KB).
+=== GCSBackupRepository
+
+Stores and retrieves backup files in a Google Cloud Storage ("GCS") bucket.
+
+GCSBackupRepsoitory accepts the following configuration options for overall 
configuration:
+
+`bucket`::
+The GCS bucket to read and write all backup files to.
+If not specified, GCSBackupRepository will use the value of the `GCS_BUCKET` 
environment variable.
+If both values are absent, the value `solrBackupsBucket` will be used as a 
default.
+
+`gcsCredentialPath`::
+A path on the local filesystem (accessible by Solr) to a 
https://cloud.google.com/iam/docs/creating-managing-service-account-keys[Google 
Cloud service account key] file.
+If not specified, GCSBackupRepository will use the value of the 
`GCS_CREDENTIAL_PATH` environment variable.
+If both values are absent, an error will be thrown as GCS requires credentials 
for most usage.
+
+`location`::
+A valid "directory" path in the given GCS bucket to us for backup strage and 
retrieval.
+(GCS uses a flat storage model, but Solr's backup functionality names blobs in 
a way that approximates hierarchical directory storage.)
+Used as a fallback when user's don't provide a `location` parameter in their 
Backup or Restore API commands
+
+In addition to these properties for overall configuration, GCSBackupRepository 
gives users detailed control over the client used to communicate with GCS.
+These properties are unlikely to interest most users, but may be valuable for 
those looking to micromanage performance or subject to a flaky network.
+
+GCSBackupRepository accepts the following advanced client-configuration 
options:
+
+`gcsClientHttpConnectTimeoutMillis`::

Review comment:
       Now we miss the configurable buffer sizes.

##########
File path: solr/solr-ref-guide/src/making-and-restoring-backups.adoc
##########
@@ -223,23 +223,190 @@ Request ID to track this action which will be processed 
asynchronously.
 
 == Backup/Restore Storage Repositories
 
-Solr provides interfaces to plug different storage systems for backing up and 
restoring. For example, you can have a Solr cluster running on a local 
filesystem like EXT3 but you can backup the indexes to a HDFS filesystem or 
vice versa.
+Solr provides a repository abstraction to allow users to backup and restore 
their data to a variety of different storage systems.

Review comment:
       Nice doc improvement!

##########
File path: 
solr/contrib/gcs-repository/src/java/org/apache/solr/gcs/GCSBackupRepository.java
##########
@@ -0,0 +1,464 @@
+/*
+ * 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.solr.gcs;
+
+import com.google.auth.oauth2.GoogleCredentials;
+import com.google.cloud.ReadChannel;
+import com.google.cloud.WriteChannel;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.BlobId;
+import com.google.cloud.storage.BlobInfo;
+import com.google.cloud.storage.Storage;
+import com.google.cloud.storage.StorageException;
+import com.google.cloud.storage.StorageOptions;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.BufferedIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.invoke.MethodHandles;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.NoSuchFileException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
+
+/**
+ * {@link BackupRepository} implementation that stores files in Google Cloud 
Storage ("GCS").
+ */
+public class GCSBackupRepository implements BackupRepository {
+    private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private static final int LARGE_BLOB_THRESHOLD_BYTE_SIZE = 5 * 1024 * 1024;
+
+    protected Storage storage;
+
+    private NamedList<Object> config = null;
+    protected String bucketName = null;
+    protected String credentialPath = null;
+    protected int writeBufferSizeBytes;
+    protected int readBufferSizeBytes;
+    protected StorageOptions.Builder storageOptionsBuilder = null;
+
+    protected Storage initStorage() {
+        if (storage != null)
+            return storage;
+
+        try {
+            if (credentialPath == null) {
+                throw new 
IllegalArgumentException(GCSConfigParser.missingCredentialErrorMsg());
+            }
+
+            log.info("Creating GCS client using credential at {}", 
credentialPath);
+            // 'GoogleCredentials.fromStream' closes the input stream, so we 
don't
+            GoogleCredentials credential = GoogleCredentials.fromStream(new 
FileInputStream(credentialPath));
+            storageOptionsBuilder.setCredentials(credential);
+            storage = storageOptionsBuilder.build().getService();
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        return storage;
+    }
+
+    @Override
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public void init(NamedList args) {
+        this.config = (NamedList<Object>) args;
+        final GCSConfigParser configReader = new GCSConfigParser();
+        final GCSConfigParser.GCSConfig parsedConfig = 
configReader.parseConfiguration(config);
+
+        this.bucketName = parsedConfig.getBucketName();
+        this.credentialPath = parsedConfig.getCredentialPath();
+        this.writeBufferSizeBytes = parsedConfig.getWriteBufferSize();
+        this.readBufferSizeBytes = parsedConfig.getReadBufferSize();
+        this.storageOptionsBuilder = parsedConfig.getStorageOptionsBuilder();
+
+        initStorage();
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> T getConfigProperty(String name) {
+        return (T) this.config.get(name);
+    }
+
+    @Override
+    public URI createURI(String location) {
+        Objects.requireNonNull(location);
+
+        URI result;
+        try {
+            result = new URI(location);
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("Error on creating URI", e);
+        }
+
+        return result;
+    }
+
+    @Override
+    public URI resolve(URI baseUri, String... pathComponents) {
+        StringBuilder builder = new StringBuilder(baseUri.toString());
+        for (String path : pathComponents) {
+            if (path != null && !path.isEmpty()) {
+                if (builder.charAt(builder.length()-1) != '/') {
+                    builder.append('/');
+                }
+                builder.append(path);
+            }
+        }
+
+        return URI.create(builder.toString());
+    }
+
+    @Override
+    public boolean exists(URI path) throws IOException {
+        if 
(path.toString().equals(getConfigProperty(CoreAdminParams.BACKUP_LOCATION))) {
+            return true;
+        }
+
+        if (path.toString().endsWith("/")) {
+            return storage.get(bucketName, path.toString(), 
Storage.BlobGetOption.fields()) != null;
+        } else {
+            final String filePath = path.toString();
+            final String directoryPath = path.toString() + "/";
+            return storage.get(bucketName, filePath, 
Storage.BlobGetOption.fields()) != null ||
+                    storage.get(bucketName, directoryPath, 
Storage.BlobGetOption.fields()) != null;
+        }
+
+    }
+
+    @Override
+    public PathType getPathType(URI path) throws IOException {
+        if (path.toString().endsWith("/"))
+            return PathType.DIRECTORY;
+
+        Blob blob = storage.get(bucketName, path.toString()+"/", 
Storage.BlobGetOption.fields());
+        if (blob != null)
+            return PathType.DIRECTORY;
+
+        return PathType.FILE;
+    }
+
+    @Override
+    public String[] listAll(URI path) throws IOException {
+        final String blobName = 
appendTrailingSeparatorIfNecessary(path.toString());
+
+        final String pathStr = blobName;
+        final LinkedList<String> result = new LinkedList<>();
+        storage.list(
+                bucketName,
+                Storage.BlobListOption.currentDirectory(),
+                Storage.BlobListOption.prefix(pathStr),
+                Storage.BlobListOption.fields())
+        .iterateAll().forEach(
+                blob -> {
+                    assert blob.getName().startsWith(pathStr);
+                    final String suffixName = 
blob.getName().substring(pathStr.length());
+                    if (!suffixName.isEmpty()) {
+                        // Remove trailing '/' if present
+                        if (suffixName.endsWith("/")) {
+                            result.add(suffixName.substring(0, 
suffixName.length() - 1));
+                        } else {
+                            result.add(suffixName);
+                        }
+                    }
+                });
+
+        return result.toArray(new String[0]);
+    }
+
+    @Override
+    public IndexInput openInput(URI dirPath, String fileName, IOContext ctx) 
throws IOException {
+        return openInput(dirPath, fileName, ctx, readBufferSizeBytes);
+    }
+
+    private IndexInput openInput(URI dirPath, String fileName, IOContext ctx, 
int bufferSize) {
+        String blobName = dirPath.toString();
+        blobName = appendTrailingSeparatorIfNecessary(blobName);
+        blobName += fileName;
+
+        final BlobId blobId = BlobId.of(bucketName, blobName);
+        final Blob blob = storage.get(blobId, 
Storage.BlobGetOption.fields(Storage.BlobField.SIZE));
+        final ReadChannel readChannel = blob.reader();
+        readChannel.setChunkSize(bufferSize);
+
+        return new BufferedIndexInput(blobName, bufferSize) {
+
+            @Override
+            public long length() {
+                return blob.getSize();
+            }
+
+            @Override
+            protected void readInternal(ByteBuffer b) throws IOException {
+                readChannel.read(b);
+            }
+
+            @Override
+            protected void seekInternal(long pos) throws IOException {
+                readChannel.seek(pos);
+            }
+
+            @Override
+            public void close() throws IOException {
+                readChannel.close();
+            }
+        };
+    }
+
+    @Override
+    public OutputStream createOutput(URI path) throws IOException {
+        final BlobInfo blobInfo = BlobInfo.newBuilder(bucketName, 
path.toString()).build();
+        final WriteChannel writeChannel = storage.writer(blobInfo, 
getDefaultBlobWriteOptions());
+
+        return Channels.newOutputStream(new WritableByteChannel() {
+            @Override
+            public int write(ByteBuffer src) throws IOException {
+                return writeChannel.write(src);
+            }
+
+            @Override
+            public boolean isOpen() {
+                return writeChannel.isOpen();
+            }
+
+            @Override
+            public void close() throws IOException {
+                writeChannel.close();
+            }
+        });
+    }
+
+    @Override
+    public void createDirectory(URI path) throws IOException {
+        final String name = 
appendTrailingSeparatorIfNecessary(path.toString());
+        storage.create(BlobInfo.newBuilder(bucketName, name).build()) ;
+    }
+
+    @Override
+    public void deleteDirectory(URI path) throws IOException {
+        List<BlobId> blobIds = allBlobsAtDir(path);
+        if (!blobIds.isEmpty()) {
+            storage.delete(blobIds);
+        } else {
+            log.debug("Path:{} doesn't have any blobs", path);
+        }
+    }
+
+    protected List<BlobId> allBlobsAtDir(URI path) throws IOException {
+        final String blobName = 
appendTrailingSeparatorIfNecessary(path.toString());
+
+        final List<BlobId> result = new ArrayList<>();
+        final String pathStr = blobName;
+        storage.list(
+                bucketName,
+                Storage.BlobListOption.prefix(pathStr),
+                Storage.BlobListOption.fields())
+        .iterateAll().forEach(
+                blob -> result.add(blob.getBlobId())
+        );
+
+        return result;
+
+    }
+
+    @Override
+    public void delete(URI path, Collection<String> files, boolean 
ignoreNoSuchFileException) throws IOException {
+        if (files.isEmpty()) {
+            return;
+        }
+        final String prefix = 
appendTrailingSeparatorIfNecessary(path.toString());
+        List<BlobId> blobDeletes = files.stream()
+                .map(file -> BlobId.of(bucketName, prefix + file))
+                .collect(Collectors.toList());
+        List<Boolean> result = storage.delete(blobDeletes);
+        if (!ignoreNoSuchFileException) {
+            int failedDelete = result.indexOf(Boolean.FALSE);
+            if (failedDelete != -1) {
+                throw new NoSuchFileException("File " + 
blobDeletes.get(failedDelete).getName() + " was not found");
+            }
+        }
+    }
+
+    @Override
+    public void copyIndexFileFrom(Directory sourceDir, String sourceFileName, 
URI destDir, String destFileName) throws IOException {
+        String blobName = destDir.toString();
+        blobName = appendTrailingSeparatorIfNecessary(blobName);
+        blobName += destFileName;
+        final BlobInfo blobInfo = BlobInfo.newBuilder(bucketName, 
blobName).build();
+        try (ChecksumIndexInput input = 
sourceDir.openChecksumInput(sourceFileName, 
DirectoryFactory.IOCONTEXT_NO_CACHE)) {
+            if (input.length() <= CodecUtil.footerLength()) {
+                throw new CorruptIndexException("file is too small:" + 
input.length(), input);
+            }
+            if (input.length() > LARGE_BLOB_THRESHOLD_BYTE_SIZE) {
+                writeBlobResumable(blobInfo, input);
+            } else {
+                writeBlobMultipart(blobInfo, input, (int) input.length());
+            }
+        }
+    }
+
+    @Override
+    public void copyIndexFileTo(URI sourceRepo, String sourceFileName, 
Directory dest, String destFileName) throws IOException {
+        try {
+            String blobName = sourceRepo.toString();
+            blobName = appendTrailingSeparatorIfNecessary(blobName);
+            blobName += sourceFileName;
+            final BlobId blobId = BlobId.of(bucketName, blobName);
+            try (final ReadChannel readChannel = storage.reader(blobId);
+                 IndexOutput output = dest.createOutput(destFileName, 
DirectoryFactory.IOCONTEXT_NO_CACHE)) {
+                ByteBuffer buffer = ByteBuffer.allocate(readBufferSizeBytes);
+                while (readChannel.read(buffer) > 0) {
+                    buffer.flip();
+                    byte[] arr = buffer.array();
+                    output.writeBytes(arr, buffer.position(), buffer.limit() - 
buffer.position());
+                    buffer.clear();
+                }
+            }
+        } catch (Exception e) {
+            log.info("Here's an exception e", e);
+        }
+    }
+
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    private void writeBlobMultipart(BlobInfo blobInfo, ChecksumIndexInput 
indexInput, int blobSize)
+            throws IOException {
+        byte[] bytes = new byte[blobSize];
+        indexInput.readBytes(bytes, 0, blobSize - CodecUtil.footerLength());
+        long checksum = CodecUtil.checkFooter(indexInput);
+        ByteBuffer footerBuffer = ByteBuffer.wrap(bytes, blobSize - 
CodecUtil.footerLength(), CodecUtil.footerLength());
+        writeFooter(checksum, footerBuffer);
+        try {
+            storage.create(blobInfo, bytes, 
Storage.BlobTargetOption.doesNotExist());
+        } catch (final StorageException se) {
+            if (se.getCode() == HTTP_PRECON_FAILED) {
+                throw new 
FileAlreadyExistsException(blobInfo.getBlobId().getName(), null, 
se.getMessage());
+            }
+            throw se;
+        }
+    }
+
+    private void writeBlobResumable(BlobInfo blobInfo, ChecksumIndexInput 
indexInput) throws IOException {
+        try {
+            final WriteChannel writeChannel = storage.writer(blobInfo, 
getDefaultBlobWriteOptions());
+
+            ByteBuffer buffer = ByteBuffer.allocate(writeBufferSizeBytes);
+            writeChannel.setChunkSize(writeBufferSizeBytes);
+
+            long remain = indexInput.length() - CodecUtil.footerLength();
+            while (remain > 0) {
+                // reading
+                int byteReads = (int) Math.min(buffer.capacity(), remain);
+                indexInput.readBytes(buffer.array(), 0, byteReads);
+                buffer.position(byteReads);
+                buffer.flip();
+
+                // writing
+                writeChannel.write(buffer);
+                buffer.clear();
+                remain -= byteReads;
+            }
+            long checksum = CodecUtil.checkFooter(indexInput);
+            ByteBuffer bytes = getFooter(checksum);
+            writeChannel.write(bytes);
+            writeChannel.close();
+        } catch (final StorageException se) {
+            if (se.getCode() == HTTP_PRECON_FAILED) {
+                throw new 
FileAlreadyExistsException(blobInfo.getBlobId().getName(), null, 
se.getMessage());
+            }
+            throw se;
+        }
+    }
+
+    private ByteBuffer getFooter(long checksum) throws IOException {
+        ByteBuffer buffer = ByteBuffer.allocate(CodecUtil.footerLength());
+        writeFooter(checksum, buffer);
+        return buffer;
+    }
+
+    private void writeFooter(long checksum, ByteBuffer buffer) throws 
IOException {
+        IndexOutput out = new IndexOutput("", "") {
+
+            @Override
+            public void writeByte(byte b) throws IOException {
+                buffer.put(b);
+            }
+
+            @Override
+            public void writeBytes(byte[] b, int offset, int length) throws 
IOException {
+                buffer.put(b, offset, length);
+            }
+
+            @Override
+            public void close() throws IOException {
+
+            }
+
+            @Override
+            public long getFilePointer() {
+                return 0;
+            }
+
+            @Override
+            public long getChecksum() throws IOException {
+                return checksum;
+            }
+        };
+        CodecUtil.writeFooter(out);
+        buffer.flip();
+    }
+
+    protected Storage.BlobWriteOption[] getDefaultBlobWriteOptions() {
+        return new Storage.BlobWriteOption[0];

Review comment:
       Could be a constant NO_WRITE_OPTIONS.

##########
File path: 
solr/contrib/gcs-repository/src/java/org/apache/solr/gcs/GCSConfigParser.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.solr.gcs;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.cloud.storage.StorageOptions;
+import org.apache.solr.common.util.NamedList;
+import org.threeten.bp.Duration;
+
+import java.util.Map;
+
+/**
+ * Parses configuration for {@link GCSBackupRepository} from NamedList and 
environment variables
+ */
+public class GCSConfigParser {
+  private static final String GCS_BUCKET_ENV_VAR_NAME = "GCS_BUCKET";
+  private static final String GCS_CREDENTIAL_ENV_VAR_NAME = 
"GCS_CREDENTIAL_PATH";
+
+  private static final String GCS_BUCKET_PARAM_NAME = "bucket";

Review comment:
       Do we want to name it 'gcsBucket'?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@solr.apache.org
For additional commands, e-mail: issues-h...@solr.apache.org

Reply via email to