You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by cpoerschke <gi...@git.apache.org> on 2014/05/13 18:42:28 UTC

[GitHub] lucene-solr pull request: solr: timing-debug to include lucene-sea...

GitHub user cpoerschke opened a pull request:

    https://github.com/apache/lucene-solr/pull/52

    solr: timing-debug to include lucene-search-time

    This pull request is for https://issues.apache.org/jira/i#browse/SOLR-6068
    
    This pull request includes https://github.com/apache/lucene-solr/pull/48 change proposed for https://issues.apache.org/jira/i#browse/SOLR-6067

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/bloomberg/lucene-solr branch_4x-time-lucene-search

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/lucene-solr/pull/52.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #52
    
----
commit 446bb651b636d7c81fcffa1c50697d953e2f2708
Author: Christine Poerschke <cp...@bloomberg.net>
Date:   2014-05-09T14:24:50Z

    solr: add buildAndRunCollectorChain method to reduce code duplication in SolrIndexSearcher

commit 8c2a4bcd81ee8c37caf0c398db8e1a80bb0f548f
Author: Christine Poerschke <cp...@bloomberg.net>
Date:   2014-05-13T15:47:32Z

    solr: timing-debug to include lucene-search-time
    
    Example input/output:
    
    curl "http://localhost:8787/solr/collection1/select?rows=0&wt=json&debug=timing&q=language_id:1"
    
    {"responseHeader":{"status":0,"QTime":69},"response":{"numFound":26,"start":0,"maxScore":0.9354614,"docs":[]},"debug":{"timing":{"time":48.0,"prepare":{"time":0.0,"query":{"time":0.0},"stats":{"time":0.0},"debug":{"time":0.0}},"process":{"time":48.0,"query":{"time":48.0,"LuceneSearchTime":10.0},"stats":{"time":0.0},"debug":{"time":0.0}}}}}

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71725905
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java ---
    @@ -919,7 +928,93 @@ public void call(CallInfo callInfo) throws Exception {
             }
           }
         }
    +  },
    +  CREATESNAPSHOT_OP(CREATESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +        if (core.getDeletionPolicy().getLatestCommit() == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find latest commit. Please ensure to execute a hard commit");
    +        }
    +
    +        String indexDirPath = core.getIndexDir();
    +        IndexCommit ic = core.getDeletionPolicy().getLatestCommit();
    +        SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
    +        mgr.snapshot(commitName, indexDirPath, ic.getGeneration());
    +      }
    +    }
    +  },
    +  DELETESNAPSHOT_OP(DELETESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +
    +        SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
    +        Optional<SnapshotMetaData> metadata = mgr.release(commitName);
    +        if (metadata.isPresent()) {
    +          long gen = metadata.get().getGenerationNumber();
    +          String indexDirPath = metadata.get().getIndexDirPath();
    +
    +          // If the directory storing the snapshot is not the same as the *current* core
    --- End diff --
    
    This explanation helps; thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71721619
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    +        .findFirst()
    +        .isPresent();
    +  }
    +
    +  /**
    +   * This method returns the snapshot meta-data for the specified name (if it exists).
    +   *
    +   * @param name The name of the snapshot
    +   * @return The snapshot meta-data if exists.
    +   */
    +  public synchronized Optional<SnapshotMetaData> getSnapshotMetaData(String name) {
    +    return Optional.ofNullable(nameToDetailsMapping.get(name));
    +  }
    +
    +  /**
    +   * @return A list of snapshots created so far.
    +   */
    +  public synchronized Collection<String> listSnapshots() {
    +    return new HashSet<>(nameToDetailsMapping.keySet());
    +  }
    +
    +  /**
    +   * This method returns a list of snapshots created in a specified index directory.
    +   *
    +   * @param indexDirPath The index directory path.
    +   * @return a list snapshots stored in the specified directory.
    +   */
    +  public synchronized Collection<SnapshotMetaData> listSnapshotsInIndexDir(String indexDirPath) {
    +    return nameToDetailsMapping.entrySet().stream()
    +        .filter(entry -> indexDirPath.equals(entry.getValue().getIndexDirPath()))
    +        .map(entry -> entry.getValue())
    +        .collect(Collectors.toList());
    +  }
    +
    +  /**
    +   * This method returns the {@linkplain IndexCommit} associated with the specified
    +   * <code>commitName</code>. A snapshot with specified <code>commitName</code> must
    +   * be created before invoking this method.
    +   *
    +   * @param commitName The name of persisted commit
    +   * @return the {@linkplain IndexCommit}
    +   * @throws IOException in case of I/O error.
    +   */
    +  public Optional<IndexCommit> getIndexCommitByName(String commitName) throws IOException {
    +    Optional<IndexCommit> result = Optional.empty();
    +    Optional<SnapshotMetaData> metaData = getSnapshotMetaData(commitName);
    +    if (metaData.isPresent()) {
    +      String indexDirPath = metaData.get().getIndexDirPath();
    +      long gen = metaData.get().getGenerationNumber();
    +
    +      Directory d = solrCore.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, DirectoryFactory.LOCK_TYPE_NONE);
    +      try {
    +        result = DirectoryReader.listCommits(d)
    +                                .stream()
    +                                .filter(ic -> ic.getGeneration() == gen)
    +                                .findAny();
    +
    +        if (!result.isPresent()) {
    +          log.warn("Unable to find commit with generation {} in the directory {}", gen, indexDirPath);
    +        }
    +
    +      } finally {
    +        solrCore.getDirectoryFactory().release(d);
    +      }
    +    } else {
    +      log.warn("Commit with name {} is not persisted for core {}", commitName, solrCore.getName());
    +    }
    +
    +    return result;
    +  }
    +
    +  synchronized private void persist() throws IOException {
    --- End diff --
    
    Nitpick: private first


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71747583
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java ---
    @@ -919,7 +928,93 @@ public void call(CallInfo callInfo) throws Exception {
             }
           }
         }
    +  },
    +  CREATESNAPSHOT_OP(CREATESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +        if (core.getDeletionPolicy().getLatestCommit() == null) {
    --- End diff --
    
    I see that ReplicationHandler is implemented differently. If the latestCommit is null, it uses the commit opened by the index searcher. Now I am using the same logic here, assuming the index searcher will always return non-null index commit. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71723294
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java ---
    @@ -512,11 +513,24 @@ private void doSnapShoot(SolrParams params, SolrQueryResponse rsp,
             numberToKeep = Integer.MAX_VALUE;
           }
     
    -      IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
    -      IndexCommit indexCommit = delPolicy.getLatestCommit();
    +      IndexCommit indexCommit = null;
    +      String commitName = params.get(CoreAdminParams.COMMIT_NAME);
    +      if (commitName != null) {
    +        SolrSnapshotMetaDataManager snapshotMgr = core.getSnapshotMetaDataManager();
    +        Optional<IndexCommit> commit = snapshotMgr.getIndexCommitByName(commitName);
    +        if(commit.isPresent()) {
    +          indexCommit = commit.get();
    +        } else {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to find an index commit with name " + commitName +
    --- End diff --
    
    I think this should be a BAD_REQUEST


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #52: [SOLR-9269] Ability to create/delete/list snapshots f...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on the issue:

    https://github.com/apache/lucene-solr/pull/52
  
    I forgot to use the magic words in my commit to auto-close this, so you can close this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71828221
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.store.Directory;
    +import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This class provides functionality required to handle the data files corresponding to Solr snapshots.
    + */
    +public class SolrSnapshotManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  /**
    +   * This method deletes index files of the {@linkplain IndexCommit} for the specified generation number.
    +   *
    +   * @param dir The index directory storing the snapshot.
    +   * @param gen The generation number for the {@linkplain IndexCommit}
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteIndexFiles ( Directory dir, Collection<SnapshotMetaData> snapshots, long gen ) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots,commits);
    +    for (IndexCommit ic : commits) {
    +      if (ic.getGeneration() == gen) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +        break;
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes all files not corresponding to a configured snapshot in the specified index directory.
    +   *
    +   * @param dir The index directory to search for.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteNonSnapshotIndexFiles (Directory dir, Collection<SnapshotMetaData> snapshots) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots, commits);
    +    Set<Long> snapshotGenNumbers = snapshots.stream()
    +                                            .map(SnapshotMetaData::getGenerationNumber)
    +                                            .collect(Collectors.toSet());
    +    for (IndexCommit ic : commits) {
    +      if (!snapshotGenNumbers.contains(ic.getGeneration())) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method computes reference count for the index files by taking into consideration
    +   * (a) configured snapshots and (b) files sharing between two or more {@linkplain IndexCommit} instances.
    +   *
    +   * @param snapshots A collection of user configured snapshots
    +   * @param commits A list of {@linkplain IndexCommit} instances
    +   * @return A map containing reference count for each index file referred in one of the {@linkplain IndexCommit} instances.
    +   * @throws IOException in case of I/O error.
    +   */
    +  public static Map<String, Integer> buildRefCounts (Collection<SnapshotMetaData> snapshots, List<IndexCommit> commits) throws IOException {
    --- End diff --
    
    Actually we need to know the actual reference count for the unit test. But now I changed this method to be package protected. Hence we should be OK with this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71745546
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    +        .findFirst()
    +        .isPresent();
    +  }
    +
    +  /**
    +   * This method returns the snapshot meta-data for the specified name (if it exists).
    +   *
    +   * @param name The name of the snapshot
    +   * @return The snapshot meta-data if exists.
    +   */
    +  public synchronized Optional<SnapshotMetaData> getSnapshotMetaData(String name) {
    +    return Optional.ofNullable(nameToDetailsMapping.get(name));
    +  }
    +
    +  /**
    +   * @return A list of snapshots created so far.
    +   */
    +  public synchronized Collection<String> listSnapshots() {
    +    return new HashSet<>(nameToDetailsMapping.keySet());
    +  }
    +
    +  /**
    +   * This method returns a list of snapshots created in a specified index directory.
    +   *
    +   * @param indexDirPath The index directory path.
    +   * @return a list snapshots stored in the specified directory.
    +   */
    +  public synchronized Collection<SnapshotMetaData> listSnapshotsInIndexDir(String indexDirPath) {
    +    return nameToDetailsMapping.entrySet().stream()
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71727810
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    +        .findFirst()
    +        .isPresent();
    +  }
    +
    +  /**
    +   * This method returns the snapshot meta-data for the specified name (if it exists).
    +   *
    +   * @param name The name of the snapshot
    +   * @return The snapshot meta-data if exists.
    +   */
    +  public synchronized Optional<SnapshotMetaData> getSnapshotMetaData(String name) {
    +    return Optional.ofNullable(nameToDetailsMapping.get(name));
    +  }
    +
    +  /**
    +   * @return A list of snapshots created so far.
    +   */
    +  public synchronized Collection<String> listSnapshots() {
    +    return new HashSet<>(nameToDetailsMapping.keySet());
    --- End diff --
    
    ArrayList for consistent ordering.  Add comment that we need to copy for thread-safety.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71730404
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    --- End diff --
    
    Since this may be called often (?) I suggest a quick nameToDetailsMapping.isEmpty() check first to return fast.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71748142
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java ---
    @@ -919,7 +928,93 @@ public void call(CallInfo callInfo) throws Exception {
             }
           }
         }
    +  },
    +  CREATESNAPSHOT_OP(CREATESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +        if (core.getDeletionPolicy().getLatestCommit() == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find latest commit. Please ensure to execute a hard commit");
    +        }
    +
    +        String indexDirPath = core.getIndexDir();
    +        IndexCommit ic = core.getDeletionPolicy().getLatestCommit();
    +        SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
    +        mgr.snapshot(commitName, indexDirPath, ic.getGeneration());
    +      }
    +    }
    +  },
    +  DELETESNAPSHOT_OP(DELETESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +
    +        SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
    +        Optional<SnapshotMetaData> metadata = mgr.release(commitName);
    +        if (metadata.isPresent()) {
    --- End diff --
    
    OK I am using BAD_REQUEST for parameter related errors. I don't think we should throw error if the commit is not found since this is in-line with the user expectation (which is Solr should not have a snapshot with the specified name).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr issue #52: [SOLR-9269] Ability to create/delete/list snapshots f...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on the issue:

    https://github.com/apache/lucene-solr/pull/52
  
    The last SolrSnapshotManager.deleteIndexFiles is still synchronized; I think you forgot this one?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71735109
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.store.Directory;
    +import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This class provides functionality required to handle the data files corresponding to Solr snapshots.
    + */
    +public class SolrSnapshotManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  /**
    +   * This method deletes index files of the {@linkplain IndexCommit} for the specified generation number.
    +   *
    +   * @param dir The index directory storing the snapshot.
    +   * @param gen The generation number for the {@linkplain IndexCommit}
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteIndexFiles ( Directory dir, Collection<SnapshotMetaData> snapshots, long gen ) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots,commits);
    +    for (IndexCommit ic : commits) {
    +      if (ic.getGeneration() == gen) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +        break;
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes all files not corresponding to a configured snapshot in the specified index directory.
    +   *
    +   * @param dir The index directory to search for.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteNonSnapshotIndexFiles (Directory dir, Collection<SnapshotMetaData> snapshots) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots, commits);
    +    Set<Long> snapshotGenNumbers = snapshots.stream()
    +                                            .map(SnapshotMetaData::getGenerationNumber)
    +                                            .collect(Collectors.toSet());
    +    for (IndexCommit ic : commits) {
    +      if (!snapshotGenNumbers.contains(ic.getGeneration())) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method computes reference count for the index files by taking into consideration
    +   * (a) configured snapshots and (b) files sharing between two or more {@linkplain IndexCommit} instances.
    +   *
    +   * @param snapshots A collection of user configured snapshots
    +   * @param commits A list of {@linkplain IndexCommit} instances
    +   * @return A map containing reference count for each index file referred in one of the {@linkplain IndexCommit} instances.
    +   * @throws IOException in case of I/O error.
    +   */
    +  public static Map<String, Integer> buildRefCounts (Collection<SnapshotMetaData> snapshots, List<IndexCommit> commits) throws IOException {
    --- End diff --
    
    It seems that the callers of this method don't actually need to know the ref counts.  The callers want to know which file names referred to.  I think returning exactly that, and naming it as such, would be much more helpful.
    (could be package-private, BTW)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71730215
  
    --- Diff: solr/core/src/java/org/apache/solr/core/SolrCore.java ---
    @@ -414,7 +416,19 @@ private IndexDeletionPolicyWrapper initDeletionPolicy(IndexDeletionPolicyWrapper
         } else {
           delPolicy = new SolrDeletionPolicy();
         }
    -    return new IndexDeletionPolicyWrapper(delPolicy);
    +
    +    return new IndexDeletionPolicyWrapper(delPolicy, snapshotMgr);
    +  }
    +
    +  private SolrSnapshotMetaDataManager initSnapshotMetaDataManager() {
    --- End diff --
    
    What do you think about possibly creating a Directory lazily on first use of a snapshot?  I'm a tad concerned we're weighing Solr down a bit more for everyone when only a small number of installations in the future will use this.  What's your feeling @markrmiller ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71719038
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    --- End diff --
    
    .anyMatch() is simpler


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71745628
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    +        .findFirst()
    +        .isPresent();
    +  }
    +
    +  /**
    +   * This method returns the snapshot meta-data for the specified name (if it exists).
    +   *
    +   * @param name The name of the snapshot
    +   * @return The snapshot meta-data if exists.
    +   */
    +  public synchronized Optional<SnapshotMetaData> getSnapshotMetaData(String name) {
    +    return Optional.ofNullable(nameToDetailsMapping.get(name));
    +  }
    +
    +  /**
    +   * @return A list of snapshots created so far.
    +   */
    +  public synchronized Collection<String> listSnapshots() {
    +    return new HashSet<>(nameToDetailsMapping.keySet());
    +  }
    +
    +  /**
    +   * This method returns a list of snapshots created in a specified index directory.
    +   *
    +   * @param indexDirPath The index directory path.
    +   * @return a list snapshots stored in the specified directory.
    +   */
    +  public synchronized Collection<SnapshotMetaData> listSnapshotsInIndexDir(String indexDirPath) {
    +    return nameToDetailsMapping.entrySet().stream()
    +        .filter(entry -> indexDirPath.equals(entry.getValue().getIndexDirPath()))
    +        .map(entry -> entry.getValue())
    +        .collect(Collectors.toList());
    +  }
    +
    +  /**
    +   * This method returns the {@linkplain IndexCommit} associated with the specified
    +   * <code>commitName</code>. A snapshot with specified <code>commitName</code> must
    +   * be created before invoking this method.
    +   *
    +   * @param commitName The name of persisted commit
    +   * @return the {@linkplain IndexCommit}
    +   * @throws IOException in case of I/O error.
    +   */
    +  public Optional<IndexCommit> getIndexCommitByName(String commitName) throws IOException {
    +    Optional<IndexCommit> result = Optional.empty();
    +    Optional<SnapshotMetaData> metaData = getSnapshotMetaData(commitName);
    +    if (metaData.isPresent()) {
    +      String indexDirPath = metaData.get().getIndexDirPath();
    +      long gen = metaData.get().getGenerationNumber();
    +
    +      Directory d = solrCore.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, DirectoryFactory.LOCK_TYPE_NONE);
    +      try {
    +        result = DirectoryReader.listCommits(d)
    +                                .stream()
    +                                .filter(ic -> ic.getGeneration() == gen)
    +                                .findAny();
    +
    +        if (!result.isPresent()) {
    +          log.warn("Unable to find commit with generation {} in the directory {}", gen, indexDirPath);
    +        }
    +
    +      } finally {
    +        solrCore.getDirectoryFactory().release(d);
    +      }
    +    } else {
    +      log.warn("Commit with name {} is not persisted for core {}", commitName, solrCore.getName());
    +    }
    +
    +    return result;
    +  }
    +
    +  synchronized private void persist() throws IOException {
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71746146
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java ---
    @@ -512,11 +513,24 @@ private void doSnapShoot(SolrParams params, SolrQueryResponse rsp,
             numberToKeep = Integer.MAX_VALUE;
           }
     
    -      IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
    -      IndexCommit indexCommit = delPolicy.getLatestCommit();
    +      IndexCommit indexCommit = null;
    +      String commitName = params.get(CoreAdminParams.COMMIT_NAME);
    +      if (commitName != null) {
    +        SolrSnapshotMetaDataManager snapshotMgr = core.getSnapshotMetaDataManager();
    +        Optional<IndexCommit> commit = snapshotMgr.getIndexCommitByName(commitName);
    +        if(commit.isPresent()) {
    +          indexCommit = commit.get();
    +        } else {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to find an index commit with name " + commitName +
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71753653
  
    --- Diff: solr/core/src/test/org/apache/solr/handler/TestSolrCoreSnapshots.java ---
    @@ -0,0 +1,446 @@
    +/*
    + * 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.handler;
    +
    +import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.lang.invoke.MethodHandles;
    +import java.net.URL;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.Paths;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import org.apache.commons.io.IOUtils;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexNotFoundException;
    +import org.apache.lucene.store.SimpleFSDirectory;
    +import org.apache.lucene.util.TestUtil;
    +import org.apache.lucene.util.LuceneTestCase.Slow;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.client.solrj.SolrClient;
    +import org.apache.solr.client.solrj.impl.CloudSolrClient;
    +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
    +import org.apache.solr.client.solrj.request.CoreAdminRequest.CreateSnapshot;
    +import org.apache.solr.client.solrj.request.CoreAdminRequest.DeleteSnapshot;
    +import org.apache.solr.client.solrj.request.CoreAdminRequest.ListSnapshots;
    +import org.apache.solr.cloud.SolrCloudTestCase;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.cloud.DocCollection;
    +import org.apache.solr.common.cloud.Replica;
    +import org.apache.solr.common.cloud.Slice;
    +import org.apache.solr.common.cloud.ZkStateReader;
    +import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
    +import org.apache.solr.common.util.NamedList;
    +import org.apache.solr.core.snapshots.SolrSnapshotManager;
    +import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +@SolrTestCaseJ4.SuppressSSL // Currently unknown why SSL does not work with this test
    +@Slow
    +public class TestSolrCoreSnapshots extends SolrCloudTestCase {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static long docsSeed; // see indexDocs()
    +
    +  @BeforeClass
    +  public static void setupClass() throws Exception {
    +    useFactory("solr.StandardDirectoryFactory");
    +    configureCluster(1)// nodes
    +        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
    +        .configure();
    +
    +    docsSeed = random().nextLong();
    +  }
    +
    +  @AfterClass
    +  public static void teardownClass() throws Exception {
    +    System.clearProperty("test.build.data");
    +    System.clearProperty("test.cache.data");
    +  }
    +
    +  @Test
    +  public void testBackupRestore() throws Exception {
    +    CloudSolrClient solrClient = cluster.getSolrClient();
    +    String collectionName = "SolrCoreSnapshots";
    +    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
    +    create.process(solrClient);
    +
    +    String location = createTempDir().toFile().getAbsolutePath();
    +    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
    +
    +    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
    +    assertEquals(1, collectionState.getActiveSlices().size());
    +    Slice shard = collectionState.getActiveSlices().iterator().next();
    +    assertEquals(1, shard.getReplicas().size());
    +    Replica replica = shard.getReplicas().iterator().next();
    +
    +    String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
    +    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
    +    String backupName = TestUtil.randomSimpleString(random(), 1, 5);
    +    String commitName = TestUtil.randomSimpleString(random(), 1, 5);
    +    String duplicateName = commitName.concat("_duplicate");
    +
    +    try (
    +        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
    +        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
    +
    +      SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
    +      // Create another snapshot referring to the same index commit to verify the
    +      // reference counting implementation during snapshot deletion.
    +      SnapshotMetaData duplicateCommit = createSnapshot(adminClient, coreName, duplicateName);
    +
    +      assertEquals (metaData.getIndexDirPath(), duplicateCommit.getIndexDirPath());
    +      assertEquals (metaData.getGenerationNumber(), duplicateCommit.getGenerationNumber());
    +
    +      // Delete all documents
    +      masterClient.deleteByQuery("*:*");
    +      masterClient.commit();
    +      BackupRestoreUtils.verifyDocs(0, cluster.getSolrClient(), collectionName);
    +
    +      // Verify that the index directory contains at least 2 index commits - one referred by the snapshots
    +      // and the other containing document deletions.
    +      {
    +        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        assertTrue(2 <= commits.size());
    +      }
    +
    +      // Backup the earlier created snapshot.
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", backupName);
    +        params.put("commitName", commitName);
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), params);
    +      }
    +
    +      // Restore the backup
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", "snapshot." + backupName);
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
    +        BackupRestoreUtils.verifyDocs(nDocs, cluster.getSolrClient(), collectionName);
    +      }
    +
    +      // Verify that the old index directory (before restore) contains only those index commits referred by snapshots.
    +      {
    +        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        assertEquals(1, commits.size());
    +        assertEquals(metaData.getGenerationNumber(), commits.get(0).getGeneration());
    +      }
    +
    +      // Delete first snapshot
    +      deleteSnapshot(adminClient, coreName, commitName);
    +
    +      // Verify that corresponding index files have NOT been deleted (due to reference counting).
    +      assertFalse(listCommits(metaData.getIndexDirPath()).isEmpty());
    +
    +      // Delete second snapshot
    +      deleteSnapshot(adminClient, coreName, duplicateCommit.getName());
    +
    +      // Verify that corresponding index files have been deleted.
    +      assertTrue(listCommits(duplicateCommit.getIndexDirPath()).isEmpty());
    +    }
    +  }
    +
    +  @Test
    +  public void testHandlingSharedIndexFiles() throws Exception {
    +    CloudSolrClient solrClient = cluster.getSolrClient();
    +    String collectionName = "SolrCoreSnapshots_IndexFileSharing";
    +    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
    +    create.process(solrClient);
    +
    +    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
    +    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
    +    assertEquals(1, collectionState.getActiveSlices().size());
    +    Slice shard = collectionState.getActiveSlices().iterator().next();
    +    assertEquals(1, shard.getReplicas().size());
    +    Replica replica = shard.getReplicas().iterator().next();
    +
    +    String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
    +    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
    +    String backupName = TestUtil.randomSimpleString(random(), 1, 5);
    +    String location = createTempDir().toFile().getAbsolutePath();
    +
    +    try (
    +        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
    +        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
    +
    +      int numTests = TestUtil.nextInt(random(), 2, 5);
    +      List<SnapshotMetaData> snapshots = new ArrayList<>(numTests);
    +
    +      // Create multiple commits and create a snapshot per commit.
    +      // This should result in Lucene reusing some of the segments for later index commits.
    +      for (int attempt=0; attempt<numTests; attempt++) {
    +        if (nDocs > 0) {
    +          //Delete a few docs
    +          int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
    +          for(int i=0; i<numDeletes; i++) {
    +            masterClient.deleteByQuery("id:" + i);
    +          }
    +        }
    +
    +        // Add a few more
    +        int moreAdds = TestUtil.nextInt(random(), 1, 100);
    +        for (int i = 0; i < moreAdds; i++) {
    +          SolrInputDocument doc = new SolrInputDocument();
    +          doc.addField("id", i + nDocs);
    +          doc.addField("name", "name = " + (i + nDocs));
    +          masterClient.add(doc);
    +        }
    +        masterClient.commit();
    +
    +        // Create a snapshot
    +        snapshots.add(createSnapshot(adminClient, coreName, "snapshot_" + attempt));
    +      }
    +
    +      // Backup the earlier created snapshot.
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", backupName);
    +        params.put("commitName", snapshots.get(0).getName());
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), params);
    +      }
    +
    +      // Restore the backup. The purpose of the restore operation is to change the *current* index directory.
    +      // This is required since we delegate the file deletion to underlying IndexDeletionPolicy in case of
    +      // *current* index directory. Hence for the purpose of this test, we want to ensure that the created
    +      // snapshots are NOT in the *current* index directory.
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", "snapshot." + backupName);
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
    +      }
    +
    +      {
    +        List<IndexCommit> commits = listCommits(snapshots.get(0).getIndexDirPath());
    +        // Check if number of index commits are > 0 to ensure index file sharing.
    +        assertTrue(commits.size() > 0);
    +        Map<String,Integer> refCounts = SolrSnapshotManager.buildRefCounts(snapshots, commits);
    +
    +        SnapshotMetaData snapshotMetaData = snapshots.get(0);
    +        Optional<IndexCommit> ic = commits.stream()
    +            .filter(entry -> entry.getGeneration() == snapshotMetaData.getGenerationNumber())
    +            .findFirst();
    +        assertTrue(ic.isPresent());
    +        Collection<String> nonSharedFiles = new ArrayList<>();
    +        Collection<String> sharedFiles = new ArrayList<>();
    +        for (String fileName : ic.get().getFileNames()) {
    +          if (refCounts.getOrDefault(fileName, 0) > 1) {
    +            sharedFiles.add(fileName);
    +          } else {
    +            nonSharedFiles.add(fileName);
    +          }
    +        }
    +
    +        // Delete snapshot
    +        deleteSnapshot(adminClient, coreName, snapshotMetaData.getName());
    +
    +        // Verify that the shared files are not deleted.
    +        for (String fileName : sharedFiles) {
    +          Path path = Paths.get(snapshotMetaData.getIndexDirPath(), fileName);
    +          assertTrue(path + " should exist.", Files.exists(path));
    +        }
    +
    +        // Verify that the non-shared files are deleted.
    +        for (String fileName : nonSharedFiles) {
    +          Path path = Paths.get(snapshotMetaData.getIndexDirPath(), fileName);
    +          assertFalse(path + " should not exist.", Files.exists(path));
    +        }
    +        }
    +      }
    +  }
    +
    +  @Test
    +  public void testIndexOptimization() throws Exception {
    +    CloudSolrClient solrClient = cluster.getSolrClient();
    +    String collectionName = "SolrCoreSnapshots_IndexOptimization";
    +    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
    +    create.process(solrClient);
    +
    +    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
    +
    +    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
    +    assertEquals(1, collectionState.getActiveSlices().size());
    +    Slice shard = collectionState.getActiveSlices().iterator().next();
    +    assertEquals(1, shard.getReplicas().size());
    +    Replica replica = shard.getReplicas().iterator().next();
    +
    +    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
    +    String commitName = TestUtil.randomSimpleString(random(), 1, 5);
    +
    +    try (
    +        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
    +        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
    +
    +      SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
    +
    +      int numTests = nDocs > 0 ? TestUtil.nextInt(random(), 1, 5) : 1;
    +      for (int attempt=0; attempt<numTests; attempt++) {
    +        //Modify existing index before we call optimize.
    +        if (nDocs > 0) {
    +          //Delete a few docs
    +          int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
    +          for(int i=0; i<numDeletes; i++) {
    +            masterClient.deleteByQuery("id:" + i);
    +          }
    +          //Add a few more
    +          int moreAdds = TestUtil.nextInt(random(), 1, 100);
    +          for (int i=0; i<moreAdds; i++) {
    +            SolrInputDocument doc = new SolrInputDocument();
    +            doc.addField("id", i + nDocs);
    +            doc.addField("name", "name = " + (i + nDocs));
    +            masterClient.add(doc);
    +          }
    +          masterClient.commit();
    +        }
    +      }
    +
    +      // Before invoking optimize command, verify that the index directory contains multiple commits (including the one we snapshotted earlier).
    +      {
    +        Collection<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        // Verify that multiple index commits are stored in this directory.
    +        assertTrue(commits.size() > 0);
    +        // Verify that the snapshot commit is present in this directory.
    +        assertTrue(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
    +      }
    +
    +      // Optimize the index.
    +      masterClient.optimize(true, true, 1);
    +
    +      // After invoking optimize command, verify that the index directory contains multiple commits (including the one we snapshotted earlier).
    +      {
    +        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        // Verify that multiple index commits are stored in this directory.
    +        assertTrue(commits.size() > 1);
    +        // Verify that the snapshot commit is present in this directory.
    +        assertTrue(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
    +      }
    +
    +      // Delete the snapshot
    +      deleteSnapshot(adminClient, coreName, metaData.getName());
    +
    +      // Add few documents. Without this the optimize command below does not take effect.
    +      {
    +        int moreAdds = TestUtil.nextInt(random(), 1, 100);
    +        for (int i=0; i<moreAdds; i++) {
    +          SolrInputDocument doc = new SolrInputDocument();
    +          doc.addField("id", i + nDocs);
    +          doc.addField("name", "name = " + (i + nDocs));
    +          masterClient.add(doc);
    +        }
    +        masterClient.commit();
    +      }
    +
    +      // Optimize the index.
    +      masterClient.optimize(true, true, 1);
    +
    +      // Verify that the index directory contains only 1 index commit (which is not the same as the snapshotted commit).
    +      Collection<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +      assertTrue(commits.size() == 1);
    +      assertFalse(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
    +    }
    +  }
    +
    +  private SnapshotMetaData createSnapshot (SolrClient adminClient, String coreName, String commitName) throws Exception {
    +    CreateSnapshot req = new CreateSnapshot(commitName);
    +    req.setCoreName(coreName);
    +    adminClient.request(req);
    +
    +    Collection<SnapshotMetaData> snapshots = listSnapshots(adminClient, coreName);
    +    Optional<SnapshotMetaData> metaData = snapshots.stream().filter(x -> commitName.equals(x.getName())).findFirst();
    +    assertTrue(metaData.isPresent());
    +
    +    return metaData.get();
    +  }
    +
    +  private void deleteSnapshot(SolrClient adminClient, String coreName, String commitName) throws Exception {
    +    DeleteSnapshot req = new DeleteSnapshot(commitName);
    +    req.setCoreName(coreName);
    +    adminClient.request(req);
    +
    +    Collection<SnapshotMetaData> snapshots = listSnapshots(adminClient, coreName);
    +    assertFalse(snapshots.stream().filter(x -> commitName.equals(x.getName())).findFirst().isPresent());
    +  }
    +
    +  private Collection<SnapshotMetaData> listSnapshots(SolrClient adminClient, String coreName) throws Exception {
    +    ListSnapshots req = new ListSnapshots();
    +    req.setCoreName(coreName);
    +    NamedList resp = adminClient.request(req);
    +    assertTrue( resp.get("snapshots") instanceof NamedList );
    +    NamedList apiResult = (NamedList) resp.get("snapshots");
    +
    +    List<SnapshotMetaData> result = new ArrayList<>(apiResult.size());
    +    for(int i = 0 ; i < apiResult.size(); i++) {
    +      String commitName = apiResult.getName(i);
    +      String indexDirPath = (String)((NamedList)apiResult.get(commitName)).get("indexDirPath");
    +      long genNumber = Long.valueOf((String)((NamedList)apiResult.get(commitName)).get("generation"));
    +      result.add(new SnapshotMetaData(commitName, indexDirPath, genNumber));
    +    }
    +    return result;
    +  }
    +
    +  private List<IndexCommit> listCommits(String directory) throws Exception {
    +    SimpleFSDirectory dir = new SimpleFSDirectory(Paths.get(directory));
    +    try {
    +      return DirectoryReader.listCommits(dir);
    +    } catch (IndexNotFoundException ex) {
    +      // This can happen when the delete snapshot functionality cleans up the index files (when the directory
    +      // storing these files is not the *current* index directory).
    +      return Collections.emptyList();
    +    }
    +  }
    +
    +  static void runCoreAdminCommand(String baseUrl, String coreName, String action, Map<String,String> params)
    --- End diff --
    
    The problem is that we don't have CoreAdminRequest representation for backup/restore functionality (may be because they are internal APIs ?). Now I have moved this method to BackupRestoreUtil class so that all backup/restore related tests can use it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71736862
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.store.Directory;
    +import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This class provides functionality required to handle the data files corresponding to Solr snapshots.
    + */
    +public class SolrSnapshotManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  /**
    +   * This method deletes index files of the {@linkplain IndexCommit} for the specified generation number.
    +   *
    +   * @param dir The index directory storing the snapshot.
    +   * @param gen The generation number for the {@linkplain IndexCommit}
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteIndexFiles ( Directory dir, Collection<SnapshotMetaData> snapshots, long gen ) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots,commits);
    +    for (IndexCommit ic : commits) {
    +      if (ic.getGeneration() == gen) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +        break;
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes all files not corresponding to a configured snapshot in the specified index directory.
    +   *
    +   * @param dir The index directory to search for.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteNonSnapshotIndexFiles (Directory dir, Collection<SnapshotMetaData> snapshots) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots, commits);
    +    Set<Long> snapshotGenNumbers = snapshots.stream()
    +                                            .map(SnapshotMetaData::getGenerationNumber)
    +                                            .collect(Collectors.toSet());
    +    for (IndexCommit ic : commits) {
    +      if (!snapshotGenNumbers.contains(ic.getGeneration())) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method computes reference count for the index files by taking into consideration
    +   * (a) configured snapshots and (b) files sharing between two or more {@linkplain IndexCommit} instances.
    +   *
    +   * @param snapshots A collection of user configured snapshots
    +   * @param commits A list of {@linkplain IndexCommit} instances
    +   * @return A map containing reference count for each index file referred in one of the {@linkplain IndexCommit} instances.
    +   * @throws IOException in case of I/O error.
    +   */
    +  public static Map<String, Integer> buildRefCounts (Collection<SnapshotMetaData> snapshots, List<IndexCommit> commits) throws IOException {
    +    Map<String, Integer> result = new HashMap<>();
    +    Map<Long, IndexCommit> commitsByGen = commits.stream().collect(
    +        Collectors.toMap(IndexCommit::getGeneration, Function.identity()));
    +
    +    for(SnapshotMetaData md : snapshots) {
    +      IndexCommit ic = commitsByGen.get(md.getGenerationNumber());
    +      if (ic != null) {
    +        Collection<String> fileNames = ic.getFileNames();
    +        for(String fileName : fileNames) {
    +          int refCount = result.getOrDefault(fileName, 0);
    +          result.put(fileName, refCount+1);
    +        }
    +      }
    +    }
    +
    +    return result;
    +  }
    +
    +  /**
    +   * This method deletes the index files associated with specified <code>indexCommit</code> provided they
    +   * are not referred by some other {@linkplain IndexCommit}.
    +   *
    +   * @param dir The index directory containing the {@linkplain IndexCommit} to be deleted.
    +   * @param refCounts A map containing reference counts for each file associated with every {@linkplain IndexCommit}
    +   *                  in the specified directory.
    +   * @param indexCommit The {@linkplain IndexCommit} whose files need to be deleted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  private static synchronized void deleteIndexFiles ( Directory dir, Map<String, Integer> refCounts, IndexCommit indexCommit ) throws IOException {
    --- End diff --
    
    The "static synchronized" concerns me for the methods in this class.  This will lock this operation for the whole JVM!  Perhaps we can modify the code in this class to be tolerant of concurrent invocations.  e.g. If we fail to delete a file and the exception is a FileNotFoundException then move on.  There is similar code in Lucene.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre closed the pull request at:

    https://github.com/apache/lucene-solr/pull/52


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71727257
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    --- End diff --
    
    For more consistency/predictability (e.g. when listing snapshots), I suggest using a LinkedHashMap


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71725843
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java ---
    @@ -919,7 +928,93 @@ public void call(CallInfo callInfo) throws Exception {
             }
           }
         }
    +  },
    +  CREATESNAPSHOT_OP(CREATESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +        if (core.getDeletionPolicy().getLatestCommit() == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find latest commit. Please ensure to execute a hard commit");
    +        }
    +
    +        String indexDirPath = core.getIndexDir();
    +        IndexCommit ic = core.getDeletionPolicy().getLatestCommit();
    +        SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
    +        mgr.snapshot(commitName, indexDirPath, ic.getGeneration());
    +      }
    +    }
    +  },
    +  DELETESNAPSHOT_OP(DELETESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +
    +        SolrSnapshotMetaDataManager mgr = core.getSnapshotMetaDataManager();
    +        Optional<SnapshotMetaData> metadata = mgr.release(commitName);
    +        if (metadata.isPresent()) {
    --- End diff --
    
    Perhaps add an error if we can't find the commit name?  REQUEST_ERROR.  And FYI some other exceptions we throw should also be REQUEST_ERROR if it's based on a param.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71737044
  
    --- Diff: solr/core/src/test/org/apache/solr/handler/TestSolrCoreSnapshots.java ---
    @@ -0,0 +1,446 @@
    +/*
    + * 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.handler;
    +
    +import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.lang.invoke.MethodHandles;
    +import java.net.URL;
    +import java.nio.file.Files;
    +import java.nio.file.Path;
    +import java.nio.file.Paths;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import org.apache.commons.io.IOUtils;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexNotFoundException;
    +import org.apache.lucene.store.SimpleFSDirectory;
    +import org.apache.lucene.util.TestUtil;
    +import org.apache.lucene.util.LuceneTestCase.Slow;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.client.solrj.SolrClient;
    +import org.apache.solr.client.solrj.impl.CloudSolrClient;
    +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
    +import org.apache.solr.client.solrj.request.CoreAdminRequest.CreateSnapshot;
    +import org.apache.solr.client.solrj.request.CoreAdminRequest.DeleteSnapshot;
    +import org.apache.solr.client.solrj.request.CoreAdminRequest.ListSnapshots;
    +import org.apache.solr.cloud.SolrCloudTestCase;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.cloud.DocCollection;
    +import org.apache.solr.common.cloud.Replica;
    +import org.apache.solr.common.cloud.Slice;
    +import org.apache.solr.common.cloud.ZkStateReader;
    +import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
    +import org.apache.solr.common.util.NamedList;
    +import org.apache.solr.core.snapshots.SolrSnapshotManager;
    +import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +@SolrTestCaseJ4.SuppressSSL // Currently unknown why SSL does not work with this test
    +@Slow
    +public class TestSolrCoreSnapshots extends SolrCloudTestCase {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static long docsSeed; // see indexDocs()
    +
    +  @BeforeClass
    +  public static void setupClass() throws Exception {
    +    useFactory("solr.StandardDirectoryFactory");
    +    configureCluster(1)// nodes
    +        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
    +        .configure();
    +
    +    docsSeed = random().nextLong();
    +  }
    +
    +  @AfterClass
    +  public static void teardownClass() throws Exception {
    +    System.clearProperty("test.build.data");
    +    System.clearProperty("test.cache.data");
    +  }
    +
    +  @Test
    +  public void testBackupRestore() throws Exception {
    +    CloudSolrClient solrClient = cluster.getSolrClient();
    +    String collectionName = "SolrCoreSnapshots";
    +    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
    +    create.process(solrClient);
    +
    +    String location = createTempDir().toFile().getAbsolutePath();
    +    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
    +
    +    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
    +    assertEquals(1, collectionState.getActiveSlices().size());
    +    Slice shard = collectionState.getActiveSlices().iterator().next();
    +    assertEquals(1, shard.getReplicas().size());
    +    Replica replica = shard.getReplicas().iterator().next();
    +
    +    String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
    +    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
    +    String backupName = TestUtil.randomSimpleString(random(), 1, 5);
    +    String commitName = TestUtil.randomSimpleString(random(), 1, 5);
    +    String duplicateName = commitName.concat("_duplicate");
    +
    +    try (
    +        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
    +        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
    +
    +      SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
    +      // Create another snapshot referring to the same index commit to verify the
    +      // reference counting implementation during snapshot deletion.
    +      SnapshotMetaData duplicateCommit = createSnapshot(adminClient, coreName, duplicateName);
    +
    +      assertEquals (metaData.getIndexDirPath(), duplicateCommit.getIndexDirPath());
    +      assertEquals (metaData.getGenerationNumber(), duplicateCommit.getGenerationNumber());
    +
    +      // Delete all documents
    +      masterClient.deleteByQuery("*:*");
    +      masterClient.commit();
    +      BackupRestoreUtils.verifyDocs(0, cluster.getSolrClient(), collectionName);
    +
    +      // Verify that the index directory contains at least 2 index commits - one referred by the snapshots
    +      // and the other containing document deletions.
    +      {
    +        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        assertTrue(2 <= commits.size());
    +      }
    +
    +      // Backup the earlier created snapshot.
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", backupName);
    +        params.put("commitName", commitName);
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), params);
    +      }
    +
    +      // Restore the backup
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", "snapshot." + backupName);
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
    +        BackupRestoreUtils.verifyDocs(nDocs, cluster.getSolrClient(), collectionName);
    +      }
    +
    +      // Verify that the old index directory (before restore) contains only those index commits referred by snapshots.
    +      {
    +        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        assertEquals(1, commits.size());
    +        assertEquals(metaData.getGenerationNumber(), commits.get(0).getGeneration());
    +      }
    +
    +      // Delete first snapshot
    +      deleteSnapshot(adminClient, coreName, commitName);
    +
    +      // Verify that corresponding index files have NOT been deleted (due to reference counting).
    +      assertFalse(listCommits(metaData.getIndexDirPath()).isEmpty());
    +
    +      // Delete second snapshot
    +      deleteSnapshot(adminClient, coreName, duplicateCommit.getName());
    +
    +      // Verify that corresponding index files have been deleted.
    +      assertTrue(listCommits(duplicateCommit.getIndexDirPath()).isEmpty());
    +    }
    +  }
    +
    +  @Test
    +  public void testHandlingSharedIndexFiles() throws Exception {
    +    CloudSolrClient solrClient = cluster.getSolrClient();
    +    String collectionName = "SolrCoreSnapshots_IndexFileSharing";
    +    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
    +    create.process(solrClient);
    +
    +    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
    +    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
    +    assertEquals(1, collectionState.getActiveSlices().size());
    +    Slice shard = collectionState.getActiveSlices().iterator().next();
    +    assertEquals(1, shard.getReplicas().size());
    +    Replica replica = shard.getReplicas().iterator().next();
    +
    +    String replicaBaseUrl = replica.getStr(BASE_URL_PROP);
    +    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
    +    String backupName = TestUtil.randomSimpleString(random(), 1, 5);
    +    String location = createTempDir().toFile().getAbsolutePath();
    +
    +    try (
    +        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
    +        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
    +
    +      int numTests = TestUtil.nextInt(random(), 2, 5);
    +      List<SnapshotMetaData> snapshots = new ArrayList<>(numTests);
    +
    +      // Create multiple commits and create a snapshot per commit.
    +      // This should result in Lucene reusing some of the segments for later index commits.
    +      for (int attempt=0; attempt<numTests; attempt++) {
    +        if (nDocs > 0) {
    +          //Delete a few docs
    +          int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
    +          for(int i=0; i<numDeletes; i++) {
    +            masterClient.deleteByQuery("id:" + i);
    +          }
    +        }
    +
    +        // Add a few more
    +        int moreAdds = TestUtil.nextInt(random(), 1, 100);
    +        for (int i = 0; i < moreAdds; i++) {
    +          SolrInputDocument doc = new SolrInputDocument();
    +          doc.addField("id", i + nDocs);
    +          doc.addField("name", "name = " + (i + nDocs));
    +          masterClient.add(doc);
    +        }
    +        masterClient.commit();
    +
    +        // Create a snapshot
    +        snapshots.add(createSnapshot(adminClient, coreName, "snapshot_" + attempt));
    +      }
    +
    +      // Backup the earlier created snapshot.
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", backupName);
    +        params.put("commitName", snapshots.get(0).getName());
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.BACKUPCORE.toString(), params);
    +      }
    +
    +      // Restore the backup. The purpose of the restore operation is to change the *current* index directory.
    +      // This is required since we delegate the file deletion to underlying IndexDeletionPolicy in case of
    +      // *current* index directory. Hence for the purpose of this test, we want to ensure that the created
    +      // snapshots are NOT in the *current* index directory.
    +      {
    +        Map<String,String> params = new HashMap<>();
    +        params.put("name", "snapshot." + backupName);
    +        params.put("location", location);
    +        runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
    +      }
    +
    +      {
    +        List<IndexCommit> commits = listCommits(snapshots.get(0).getIndexDirPath());
    +        // Check if number of index commits are > 0 to ensure index file sharing.
    +        assertTrue(commits.size() > 0);
    +        Map<String,Integer> refCounts = SolrSnapshotManager.buildRefCounts(snapshots, commits);
    +
    +        SnapshotMetaData snapshotMetaData = snapshots.get(0);
    +        Optional<IndexCommit> ic = commits.stream()
    +            .filter(entry -> entry.getGeneration() == snapshotMetaData.getGenerationNumber())
    +            .findFirst();
    +        assertTrue(ic.isPresent());
    +        Collection<String> nonSharedFiles = new ArrayList<>();
    +        Collection<String> sharedFiles = new ArrayList<>();
    +        for (String fileName : ic.get().getFileNames()) {
    +          if (refCounts.getOrDefault(fileName, 0) > 1) {
    +            sharedFiles.add(fileName);
    +          } else {
    +            nonSharedFiles.add(fileName);
    +          }
    +        }
    +
    +        // Delete snapshot
    +        deleteSnapshot(adminClient, coreName, snapshotMetaData.getName());
    +
    +        // Verify that the shared files are not deleted.
    +        for (String fileName : sharedFiles) {
    +          Path path = Paths.get(snapshotMetaData.getIndexDirPath(), fileName);
    +          assertTrue(path + " should exist.", Files.exists(path));
    +        }
    +
    +        // Verify that the non-shared files are deleted.
    +        for (String fileName : nonSharedFiles) {
    +          Path path = Paths.get(snapshotMetaData.getIndexDirPath(), fileName);
    +          assertFalse(path + " should not exist.", Files.exists(path));
    +        }
    +        }
    +      }
    +  }
    +
    +  @Test
    +  public void testIndexOptimization() throws Exception {
    +    CloudSolrClient solrClient = cluster.getSolrClient();
    +    String collectionName = "SolrCoreSnapshots_IndexOptimization";
    +    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
    +    create.process(solrClient);
    +
    +    int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
    +
    +    DocCollection collectionState = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
    +    assertEquals(1, collectionState.getActiveSlices().size());
    +    Slice shard = collectionState.getActiveSlices().iterator().next();
    +    assertEquals(1, shard.getReplicas().size());
    +    Replica replica = shard.getReplicas().iterator().next();
    +
    +    String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
    +    String commitName = TestUtil.randomSimpleString(random(), 1, 5);
    +
    +    try (
    +        SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
    +        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
    +
    +      SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
    +
    +      int numTests = nDocs > 0 ? TestUtil.nextInt(random(), 1, 5) : 1;
    +      for (int attempt=0; attempt<numTests; attempt++) {
    +        //Modify existing index before we call optimize.
    +        if (nDocs > 0) {
    +          //Delete a few docs
    +          int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
    +          for(int i=0; i<numDeletes; i++) {
    +            masterClient.deleteByQuery("id:" + i);
    +          }
    +          //Add a few more
    +          int moreAdds = TestUtil.nextInt(random(), 1, 100);
    +          for (int i=0; i<moreAdds; i++) {
    +            SolrInputDocument doc = new SolrInputDocument();
    +            doc.addField("id", i + nDocs);
    +            doc.addField("name", "name = " + (i + nDocs));
    +            masterClient.add(doc);
    +          }
    +          masterClient.commit();
    +        }
    +      }
    +
    +      // Before invoking optimize command, verify that the index directory contains multiple commits (including the one we snapshotted earlier).
    +      {
    +        Collection<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        // Verify that multiple index commits are stored in this directory.
    +        assertTrue(commits.size() > 0);
    +        // Verify that the snapshot commit is present in this directory.
    +        assertTrue(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
    +      }
    +
    +      // Optimize the index.
    +      masterClient.optimize(true, true, 1);
    +
    +      // After invoking optimize command, verify that the index directory contains multiple commits (including the one we snapshotted earlier).
    +      {
    +        List<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +        // Verify that multiple index commits are stored in this directory.
    +        assertTrue(commits.size() > 1);
    +        // Verify that the snapshot commit is present in this directory.
    +        assertTrue(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
    +      }
    +
    +      // Delete the snapshot
    +      deleteSnapshot(adminClient, coreName, metaData.getName());
    +
    +      // Add few documents. Without this the optimize command below does not take effect.
    +      {
    +        int moreAdds = TestUtil.nextInt(random(), 1, 100);
    +        for (int i=0; i<moreAdds; i++) {
    +          SolrInputDocument doc = new SolrInputDocument();
    +          doc.addField("id", i + nDocs);
    +          doc.addField("name", "name = " + (i + nDocs));
    +          masterClient.add(doc);
    +        }
    +        masterClient.commit();
    +      }
    +
    +      // Optimize the index.
    +      masterClient.optimize(true, true, 1);
    +
    +      // Verify that the index directory contains only 1 index commit (which is not the same as the snapshotted commit).
    +      Collection<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
    +      assertTrue(commits.size() == 1);
    +      assertFalse(commits.stream().filter(x -> x.getGeneration() == metaData.getGenerationNumber()).findFirst().isPresent());
    +    }
    +  }
    +
    +  private SnapshotMetaData createSnapshot (SolrClient adminClient, String coreName, String commitName) throws Exception {
    +    CreateSnapshot req = new CreateSnapshot(commitName);
    +    req.setCoreName(coreName);
    +    adminClient.request(req);
    +
    +    Collection<SnapshotMetaData> snapshots = listSnapshots(adminClient, coreName);
    +    Optional<SnapshotMetaData> metaData = snapshots.stream().filter(x -> commitName.equals(x.getName())).findFirst();
    +    assertTrue(metaData.isPresent());
    +
    +    return metaData.get();
    +  }
    +
    +  private void deleteSnapshot(SolrClient adminClient, String coreName, String commitName) throws Exception {
    +    DeleteSnapshot req = new DeleteSnapshot(commitName);
    +    req.setCoreName(coreName);
    +    adminClient.request(req);
    +
    +    Collection<SnapshotMetaData> snapshots = listSnapshots(adminClient, coreName);
    +    assertFalse(snapshots.stream().filter(x -> commitName.equals(x.getName())).findFirst().isPresent());
    +  }
    +
    +  private Collection<SnapshotMetaData> listSnapshots(SolrClient adminClient, String coreName) throws Exception {
    +    ListSnapshots req = new ListSnapshots();
    +    req.setCoreName(coreName);
    +    NamedList resp = adminClient.request(req);
    +    assertTrue( resp.get("snapshots") instanceof NamedList );
    +    NamedList apiResult = (NamedList) resp.get("snapshots");
    +
    +    List<SnapshotMetaData> result = new ArrayList<>(apiResult.size());
    +    for(int i = 0 ; i < apiResult.size(); i++) {
    +      String commitName = apiResult.getName(i);
    +      String indexDirPath = (String)((NamedList)apiResult.get(commitName)).get("indexDirPath");
    +      long genNumber = Long.valueOf((String)((NamedList)apiResult.get(commitName)).get("generation"));
    +      result.add(new SnapshotMetaData(commitName, indexDirPath, genNumber));
    +    }
    +    return result;
    +  }
    +
    +  private List<IndexCommit> listCommits(String directory) throws Exception {
    +    SimpleFSDirectory dir = new SimpleFSDirectory(Paths.get(directory));
    +    try {
    +      return DirectoryReader.listCommits(dir);
    +    } catch (IndexNotFoundException ex) {
    +      // This can happen when the delete snapshot functionality cleans up the index files (when the directory
    +      // storing these files is not the *current* index directory).
    +      return Collections.emptyList();
    +    }
    +  }
    +
    +  static void runCoreAdminCommand(String baseUrl, String coreName, String action, Map<String,String> params)
    --- End diff --
    
    It's a shame you need this method... you could add the convenience methods in CoreAdminRequest like there is for other stuff


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71750235
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71828065
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotManager.java ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.store.Directory;
    +import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager.SnapshotMetaData;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This class provides functionality required to handle the data files corresponding to Solr snapshots.
    + */
    +public class SolrSnapshotManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  /**
    +   * This method deletes index files of the {@linkplain IndexCommit} for the specified generation number.
    +   *
    +   * @param dir The index directory storing the snapshot.
    +   * @param gen The generation number for the {@linkplain IndexCommit}
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteIndexFiles ( Directory dir, Collection<SnapshotMetaData> snapshots, long gen ) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots,commits);
    +    for (IndexCommit ic : commits) {
    +      if (ic.getGeneration() == gen) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +        break;
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes all files not corresponding to a configured snapshot in the specified index directory.
    +   *
    +   * @param dir The index directory to search for.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public static synchronized void deleteNonSnapshotIndexFiles (Directory dir, Collection<SnapshotMetaData> snapshots) throws IOException {
    +    List<IndexCommit> commits = DirectoryReader.listCommits(dir);
    +    Map<String, Integer> refCounts = buildRefCounts(snapshots, commits);
    +    Set<Long> snapshotGenNumbers = snapshots.stream()
    +                                            .map(SnapshotMetaData::getGenerationNumber)
    +                                            .collect(Collectors.toSet());
    +    for (IndexCommit ic : commits) {
    +      if (!snapshotGenNumbers.contains(ic.getGeneration())) {
    +        deleteIndexFiles(dir,refCounts, ic);
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method computes reference count for the index files by taking into consideration
    +   * (a) configured snapshots and (b) files sharing between two or more {@linkplain IndexCommit} instances.
    +   *
    +   * @param snapshots A collection of user configured snapshots
    +   * @param commits A list of {@linkplain IndexCommit} instances
    +   * @return A map containing reference count for each index file referred in one of the {@linkplain IndexCommit} instances.
    +   * @throws IOException in case of I/O error.
    +   */
    +  public static Map<String, Integer> buildRefCounts (Collection<SnapshotMetaData> snapshots, List<IndexCommit> commits) throws IOException {
    +    Map<String, Integer> result = new HashMap<>();
    +    Map<Long, IndexCommit> commitsByGen = commits.stream().collect(
    +        Collectors.toMap(IndexCommit::getGeneration, Function.identity()));
    +
    +    for(SnapshotMetaData md : snapshots) {
    +      IndexCommit ic = commitsByGen.get(md.getGenerationNumber());
    +      if (ic != null) {
    +        Collection<String> fileNames = ic.getFileNames();
    +        for(String fileName : fileNames) {
    +          int refCount = result.getOrDefault(fileName, 0);
    +          result.put(fileName, refCount+1);
    +        }
    +      }
    +    }
    +
    +    return result;
    +  }
    +
    +  /**
    +   * This method deletes the index files associated with specified <code>indexCommit</code> provided they
    +   * are not referred by some other {@linkplain IndexCommit}.
    +   *
    +   * @param dir The index directory containing the {@linkplain IndexCommit} to be deleted.
    +   * @param refCounts A map containing reference counts for each file associated with every {@linkplain IndexCommit}
    +   *                  in the specified directory.
    +   * @param indexCommit The {@linkplain IndexCommit} whose files need to be deleted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  private static synchronized void deleteIndexFiles ( Directory dir, Map<String, Integer> refCounts, IndexCommit indexCommit ) throws IOException {
    --- End diff --
    
    Good catch! I was mainly concerned about concurrent deletion of files e.g. due to simultaneous invocation of restore functionality. But after reviewing ReplicationHandler code and more thinking, I don't think we need to worry about it. Hence removed the synchronization.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71725010
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java ---
    @@ -919,7 +928,93 @@ public void call(CallInfo callInfo) throws Exception {
             }
           }
         }
    +  },
    +  CREATESNAPSHOT_OP(CREATESNAPSHOT) {
    +    @Override
    +    public void call(CallInfo callInfo) throws Exception {
    +      CoreContainer cc = callInfo.handler.getCoreContainer();
    +      final SolrParams params = callInfo.req.getParams();
    +
    +      String commitName = params.required().get(CoreAdminParams.COMMIT_NAME);
    +      String cname = params.required().get(CoreAdminParams.CORE);
    +      try (SolrCore core = cc.getCore(cname)) {
    +        if (core == null) {
    +          throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to locate core " + cname);
    +        }
    +        if (core.getDeletionPolicy().getLatestCommit() == null) {
    --- End diff --
    
    Firstly, this check can go below after we get it on line 949; why get it twice.  Secondly... I wonder if it's null in an initial empty-index case?  If it is, this error message is wrong/confusing... and perhaps we can support that or no?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71750261
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    +        .findFirst()
    +        .isPresent();
    +  }
    +
    +  /**
    +   * This method returns the snapshot meta-data for the specified name (if it exists).
    +   *
    +   * @param name The name of the snapshot
    +   * @return The snapshot meta-data if exists.
    +   */
    +  public synchronized Optional<SnapshotMetaData> getSnapshotMetaData(String name) {
    +    return Optional.ofNullable(nameToDetailsMapping.get(name));
    +  }
    +
    +  /**
    +   * @return A list of snapshots created so far.
    +   */
    +  public synchronized Collection<String> listSnapshots() {
    +    return new HashSet<>(nameToDetailsMapping.keySet());
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71745330
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71721509
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    +  /** Used to figure out the *current* index data directory path */
    +  private final SolrCore solrCore;
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data should be stored. Enables updating
    +   *            the existing meta-data.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir) throws IOException {
    +    this(solrCore, dir, OpenMode.CREATE_OR_APPEND);
    +  }
    +
    +  /**
    +   * A constructor.
    +   *
    +   * @param dir The directory where the snapshot meta-data is stored.
    +   * @param mode CREATE If previous meta-data should be erased.
    +   *             APPEND If previous meta-data should be read and updated.
    +   *             CREATE_OR_APPEND Creates a new meta-data structure if one does not exist
    +   *                              Updates the existing structure if one exists.
    +   * @throws IOException in case of errors.
    +   */
    +  public SolrSnapshotMetaDataManager(SolrCore solrCore, Directory dir, OpenMode mode) throws IOException {
    +    this.solrCore = solrCore;
    +    this.dir = dir;
    +
    +    if (mode == OpenMode.CREATE) {
    +      deleteSnapshotMetadataFiles();
    +    }
    +
    +    loadFromSnapshotMetadataFile();
    +
    +    if (mode == OpenMode.APPEND && nextWriteGen == 0) {
    +      throw new IllegalStateException("no snapshots stored in this directory");
    +    }
    +  }
    +
    +  /**
    +   * @return The snapshot meta-data directory
    +   */
    +  public Directory getSnapshotsDir() {
    +    return dir;
    +  }
    +
    +  /**
    +   * This method creates a new snapshot meta-data entry.
    +   *
    +   * @param name The name of the snapshot.
    +   * @param indexDirPath The directory path where the index files are stored.
    +   * @param gen The generation number for the {@linkplain IndexCommit} being snapshotted.
    +   * @throws IOException in case of I/O errors.
    +   */
    +  public synchronized void snapshot(String name, String indexDirPath, long gen) throws IOException {
    +    Preconditions.checkNotNull(name);
    +
    +    log.info("Creating the snapshot named {} for core {} associated with index commit with generation {} in directory {}"
    +        , name, solrCore.getName(), gen, indexDirPath);
    +
    +    if(nameToDetailsMapping.containsKey(name)) {
    +      throw new SolrException(ErrorCode.BAD_REQUEST, "A snapshot with name " + name + " already exists");
    +    }
    +
    +    SnapshotMetaData d = new SnapshotMetaData(name, indexDirPath, gen);
    +    nameToDetailsMapping.put(name, d);
    +
    +    boolean success = false;
    +    try {
    +      persist();
    +      success = true;
    +    } finally {
    +      if (!success) {
    +        try {
    +          release(name);
    +        } catch (Exception e) {
    +          // Suppress so we keep throwing original exception
    +        }
    +      }
    +    }
    +  }
    +
    +  /**
    +   * This method deletes a previously created snapshot (if any).
    +   *
    +   * @param name The name of the snapshot to be deleted.
    +   * @return The snapshot meta-data if the snapshot with the snapshot name exists.
    +   * @throws IOException in case of I/O error
    +   */
    +  public synchronized Optional<SnapshotMetaData> release(String name) throws IOException {
    +    log.info("Deleting the snapshot named {} for core {}", name, solrCore.getName());
    +    SnapshotMetaData result = nameToDetailsMapping.remove(Preconditions.checkNotNull(name));
    +    if(result != null) {
    +      boolean success = false;
    +      try {
    +        persist();
    +        success = true;
    +      } finally {
    +        if (!success) {
    +          nameToDetailsMapping.put(name, result);
    +        }
    +      }
    +    }
    +    return Optional.ofNullable(result);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the *current* index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(long genNumber) {
    +    return isSnapshotted(solrCore.getIndexDir(), genNumber);
    +  }
    +
    +  /**
    +   * This method returns if snapshot is created for the specified generation number in
    +   * the specified index directory.
    +   *
    +   * @param genNumber The generation number for the {@linkplain IndexCommit} to be checked.
    +   * @return true if the snapshot is created.
    +   *         false otherwise.
    +   */
    +  public synchronized boolean isSnapshotted(String indexDirPath, long genNumber) {
    +    return nameToDetailsMapping.values().stream()
    +        .filter(entry -> entry.getIndexDirPath().equals(indexDirPath) && entry.getGenerationNumber() == genNumber)
    +        .findFirst()
    +        .isPresent();
    +  }
    +
    +  /**
    +   * This method returns the snapshot meta-data for the specified name (if it exists).
    +   *
    +   * @param name The name of the snapshot
    +   * @return The snapshot meta-data if exists.
    +   */
    +  public synchronized Optional<SnapshotMetaData> getSnapshotMetaData(String name) {
    +    return Optional.ofNullable(nameToDetailsMapping.get(name));
    +  }
    +
    +  /**
    +   * @return A list of snapshots created so far.
    +   */
    +  public synchronized Collection<String> listSnapshots() {
    +    return new HashSet<>(nameToDetailsMapping.keySet());
    +  }
    +
    +  /**
    +   * This method returns a list of snapshots created in a specified index directory.
    +   *
    +   * @param indexDirPath The index directory path.
    +   * @return a list snapshots stored in the specified directory.
    +   */
    +  public synchronized Collection<SnapshotMetaData> listSnapshotsInIndexDir(String indexDirPath) {
    +    return nameToDetailsMapping.entrySet().stream()
    --- End diff --
    
    in seems you don't need the entrySet (both key and value), you only need the values()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71748418
  
    --- Diff: solr/core/src/java/org/apache/solr/core/snapshots/SolrSnapshotMetaDataManager.java ---
    @@ -0,0 +1,419 @@
    +/*
    + * 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.core.snapshots;
    +
    +import java.io.IOException;
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import org.apache.lucene.codecs.CodecUtil;
    +import org.apache.lucene.index.DirectoryReader;
    +import org.apache.lucene.index.IndexCommit;
    +import org.apache.lucene.index.IndexDeletionPolicy;
    +import org.apache.lucene.index.IndexWriterConfig.OpenMode;
    +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.lucene.util.IOUtils;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrException.ErrorCode;
    +import org.apache.solr.core.DirectoryFactory;
    +import org.apache.solr.core.IndexDeletionPolicyWrapper;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.core.DirectoryFactory.DirContext;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +/**
    + * This class is responsible to manage the persistent snapshots meta-data for the Solr indexes. The
    + * persistent snapshots are implemented by relying on Lucene {@linkplain IndexDeletionPolicy}
    + * abstraction to configure a specific {@linkplain IndexCommit} to be retained. The
    + * {@linkplain IndexDeletionPolicyWrapper} in Solr uses this class to create/delete the Solr index
    + * snapshots.
    + */
    +public class SolrSnapshotMetaDataManager {
    +  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  public static final String SNAPSHOT_METADATA_DIR = "snapshot_metadata";
    +
    +  /**
    +   * A class defining the meta-data for a specific snapshot.
    +   */
    +  public static class SnapshotMetaData {
    +    private String name;
    +    private String indexDirPath;
    +    private long generationNumber;
    +
    +    public SnapshotMetaData(String name, String indexDirPath, long generationNumber) {
    +      super();
    +      this.name = name;
    +      this.indexDirPath = indexDirPath;
    +      this.generationNumber = generationNumber;
    +    }
    +
    +    public String getName() {
    +      return name;
    +    }
    +
    +    public String getIndexDirPath() {
    +      return indexDirPath;
    +    }
    +
    +    public long getGenerationNumber() {
    +      return generationNumber;
    +    }
    +
    +    @Override
    +    public String toString() {
    +      StringBuilder builder = new StringBuilder();
    +      builder.append("SnapshotMetaData[name=");
    +      builder.append(name);
    +      builder.append(", indexDirPath=");
    +      builder.append(indexDirPath);
    +      builder.append(", generation=");
    +      builder.append(generationNumber);
    +      builder.append("]");
    +      return builder.toString();
    +    }
    +  }
    +
    +  /** Prefix used for the save file. */
    +  public static final String SNAPSHOTS_PREFIX = "snapshots_";
    +  private static final int VERSION_START = 0;
    +  private static final int VERSION_CURRENT = VERSION_START;
    +  private static final String CODEC_NAME = "solr-snapshots";
    +
    +  // The index writer which maintains the snapshots metadata
    +  private long nextWriteGen;
    +
    +  private final Directory dir;
    +
    +  /** Used to map snapshot name to snapshot meta-data. */
    +  protected final Map<String,SnapshotMetaData> nameToDetailsMapping = new HashMap<>();
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[GitHub] lucene-solr pull request #52: [SOLR-9269] Ability to create/delete/list snap...

Posted by hgadre <gi...@git.apache.org>.
Github user hgadre commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/52#discussion_r71827952
  
    --- Diff: solr/core/src/java/org/apache/solr/core/SolrCore.java ---
    @@ -414,7 +416,19 @@ private IndexDeletionPolicyWrapper initDeletionPolicy(IndexDeletionPolicyWrapper
         } else {
           delPolicy = new SolrDeletionPolicy();
         }
    -    return new IndexDeletionPolicyWrapper(delPolicy);
    +
    +    return new IndexDeletionPolicyWrapper(delPolicy, snapshotMgr);
    +  }
    +
    +  private SolrSnapshotMetaDataManager initSnapshotMetaDataManager() {
    --- End diff --
    
    I don't think directory creation should have any significant performance impact. Also lazy creation just adds a bit of complexity in the code which I wanted to avoid. But I am open to this idea if you insist.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org