summaryrefslogtreecommitdiffstats
path: root/tools/gfind_missing_files
diff options
context:
space:
mode:
Diffstat (limited to 'tools/gfind_missing_files')
-rw-r--r--tools/gfind_missing_files/Makefile.am32
-rw-r--r--tools/gfind_missing_files/gcrawler.c581
-rw-r--r--tools/gfind_missing_files/gfid_to_path.py162
-rw-r--r--tools/gfind_missing_files/gfid_to_path.sh43
-rw-r--r--tools/gfind_missing_files/gfind_missing_files.sh119
5 files changed, 937 insertions, 0 deletions
diff --git a/tools/gfind_missing_files/Makefile.am b/tools/gfind_missing_files/Makefile.am
new file mode 100644
index 00000000000..181fe7091f3
--- /dev/null
+++ b/tools/gfind_missing_files/Makefile.am
@@ -0,0 +1,32 @@
+gfindmissingfilesdir = $(GLUSTERFS_LIBEXECDIR)/gfind_missing_files
+
+if WITH_SERVER
+gfindmissingfiles_SCRIPTS = gfind_missing_files.sh gfid_to_path.sh \
+ gfid_to_path.py
+endif
+
+EXTRA_DIST = gfind_missing_files.sh gfid_to_path.sh \
+ gfid_to_path.py
+
+if WITH_SERVER
+gfindmissingfiles_PROGRAMS = gcrawler
+endif
+
+gcrawler_SOURCES = gcrawler.c
+gcrawler_LDADD = $(top_builddir)/libglusterfs/src/libglusterfs.la
+gcrawler_LDFLAGS = $(GF_LDFLAGS)
+
+AM_CPPFLAGS = $(GF_CPPFLAGS) -I$(top_srcdir)/libglusterfs/src
+
+AM_CFLAGS = -Wall $(GF_CFLAGS)
+
+if WITH_SERVER
+uninstall-local:
+ rm -f $(DESTDIR)$(sbindir)/gfind_missing_files
+
+install-data-local:
+ rm -f $(DESTDIR)$(sbindir)/gfind_missing_files
+ ln -s $(GLUSTERFS_LIBEXECDIR)/gfind_missing_files/gfind_missing_files.sh $(DESTDIR)$(sbindir)/gfind_missing_files
+endif
+
+CLEANFILES =
diff --git a/tools/gfind_missing_files/gcrawler.c b/tools/gfind_missing_files/gcrawler.c
new file mode 100644
index 00000000000..4acbe92bc8f
--- /dev/null
+++ b/tools/gfind_missing_files/gcrawler.c
@@ -0,0 +1,581 @@
+/*
+ Copyright (c) 2015 Red Hat, Inc. <http://www.redhat.com>
+ This file is part of GlusterFS.
+
+ This file is licensed to you under your choice of the GNU Lesser
+ General Public License, version 3 or any later version (LGPLv3 or
+ later), or the GNU General Public License, version 2 (GPLv2), in all
+ cases as published by the Free Software Foundation.
+*/
+
+#include <stdio.h>
+#include <errno.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <stdlib.h>
+#include <string.h>
+#include <dirent.h>
+#include <assert.h>
+#include <glusterfs/locking.h>
+
+#include <glusterfs/compat.h>
+#include <glusterfs/list.h>
+#include <glusterfs/syscall.h>
+
+#define THREAD_MAX 32
+#define BUMP(name) INC(name, 1)
+#define DEFAULT_WORKERS 4
+
+#define NEW(x) \
+ { \
+ x = calloc(1, sizeof(typeof(*x))); \
+ }
+
+#define err(x...) fprintf(stderr, x)
+#define out(x...) fprintf(stdout, x)
+#define dbg(x...) \
+ do { \
+ if (debug) \
+ fprintf(stdout, x); \
+ } while (0)
+#define tout(x...) \
+ do { \
+ out("[%ld] ", pthread_self()); \
+ out(x); \
+ } while (0)
+#define terr(x...) \
+ do { \
+ err("[%ld] ", pthread_self()); \
+ err(x); \
+ } while (0)
+#define tdbg(x...) \
+ do { \
+ dbg("[%ld] ", pthread_self()); \
+ dbg(x); \
+ } while (0)
+
+int debug = 0;
+const char *slavemnt = NULL;
+int workers = 0;
+
+struct stats {
+ unsigned long long int cnt_skipped_gfids;
+};
+
+pthread_spinlock_t stats_lock;
+
+struct stats stats_total;
+int stats = 0;
+
+#define INC(name, val) \
+ do { \
+ if (!stats) \
+ break; \
+ pthread_spin_lock(&stats_lock); \
+ { \
+ stats_total.cnt_##name += val; \
+ } \
+ pthread_spin_unlock(&stats_lock); \
+ } while (0)
+
+void
+stats_dump()
+{
+ if (!stats)
+ return;
+
+ out("-------------------------------------------\n");
+ out("Skipped_Files : %10lld\n", stats_total.cnt_skipped_gfids);
+ out("-------------------------------------------\n");
+}
+
+struct dirjob {
+ struct list_head list;
+
+ char *dirname;
+
+ struct dirjob *parent;
+ int ret; /* final status of this subtree */
+ int refcnt; /* how many dirjobs have this as parent */
+
+ pthread_spinlock_t lock;
+};
+
+struct xwork {
+ pthread_t cthreads[THREAD_MAX]; /* crawler threads */
+ int count;
+ int idle;
+ int stop;
+
+ struct dirjob crawl;
+
+ struct dirjob *rootjob; /* to verify completion in xwork_fini() */
+
+ pthread_mutex_t mutex;
+ pthread_cond_t cond;
+};
+
+struct dirjob *
+dirjob_ref(struct dirjob *job)
+{
+ pthread_spin_lock(&job->lock);
+ {
+ job->refcnt++;
+ }
+ pthread_spin_unlock(&job->lock);
+
+ return job;
+}
+
+void
+dirjob_free(struct dirjob *job)
+{
+ assert(list_empty(&job->list));
+
+ pthread_spin_destroy(&job->lock);
+ free(job->dirname);
+ free(job);
+}
+
+void
+dirjob_ret(struct dirjob *job, int err)
+{
+ int ret = 0;
+ int refcnt = 0;
+ struct dirjob *parent = NULL;
+
+ pthread_spin_lock(&job->lock);
+ {
+ refcnt = --job->refcnt;
+ job->ret = (job->ret || err);
+ }
+ pthread_spin_unlock(&job->lock);
+
+ if (refcnt == 0) {
+ ret = job->ret;
+
+ if (ret)
+ terr("Failed: %s (%d)\n", job->dirname, ret);
+ else
+ tdbg("Finished: %s\n", job->dirname);
+
+ parent = job->parent;
+ if (parent)
+ dirjob_ret(parent, ret);
+
+ dirjob_free(job);
+ job = NULL;
+ }
+}
+
+struct dirjob *
+dirjob_new(const char *dir, struct dirjob *parent)
+{
+ struct dirjob *job = NULL;
+
+ NEW(job);
+ if (!job)
+ return NULL;
+
+ job->dirname = strdup(dir);
+ if (!job->dirname) {
+ free(job);
+ return NULL;
+ }
+
+ INIT_LIST_HEAD(&job->list);
+ pthread_spin_init(&job->lock, PTHREAD_PROCESS_PRIVATE);
+ job->ret = 0;
+
+ if (parent)
+ job->parent = dirjob_ref(parent);
+
+ job->refcnt = 1;
+
+ return job;
+}
+
+void
+xwork_addcrawl(struct xwork *xwork, struct dirjob *job)
+{
+ pthread_mutex_lock(&xwork->mutex);
+ {
+ list_add_tail(&job->list, &xwork->crawl.list);
+ pthread_cond_broadcast(&xwork->cond);
+ }
+ pthread_mutex_unlock(&xwork->mutex);
+}
+
+int
+xwork_add(struct xwork *xwork, const char *dir, struct dirjob *parent)
+{
+ struct dirjob *job = NULL;
+
+ job = dirjob_new(dir, parent);
+ if (!job)
+ return -1;
+
+ xwork_addcrawl(xwork, job);
+
+ return 0;
+}
+
+struct dirjob *
+xwork_pick(struct xwork *xwork, int block)
+{
+ struct dirjob *job = NULL;
+ struct list_head *head = NULL;
+
+ head = &xwork->crawl.list;
+
+ pthread_mutex_lock(&xwork->mutex);
+ {
+ for (;;) {
+ if (xwork->stop)
+ break;
+
+ if (!list_empty(head)) {
+ job = list_entry(head->next, typeof(*job), list);
+ list_del_init(&job->list);
+ break;
+ }
+
+ if (((xwork->count * 2) == xwork->idle) &&
+ list_empty(&xwork->crawl.list)) {
+ /* no outstanding jobs, and no
+ active workers
+ */
+ tdbg("Jobless. Terminating\n");
+ xwork->stop = 1;
+ pthread_cond_broadcast(&xwork->cond);
+ break;
+ }
+
+ if (!block)
+ break;
+
+ xwork->idle++;
+ pthread_cond_wait(&xwork->cond, &xwork->mutex);
+ xwork->idle--;
+ }
+ }
+ pthread_mutex_unlock(&xwork->mutex);
+
+ return job;
+}
+
+int
+skip_name(const char *dirname, const char *name)
+{
+ if (strcmp(name, ".") == 0)
+ return 1;
+
+ if (strcmp(name, "..") == 0)
+ return 1;
+
+ if (strcmp(name, "changelogs") == 0)
+ return 1;
+
+ if (strcmp(name, "health_check") == 0)
+ return 1;
+
+ if (strcmp(name, "indices") == 0)
+ return 1;
+
+ if (strcmp(name, "landfill") == 0)
+ return 1;
+
+ return 0;
+}
+
+int
+skip_stat(struct dirjob *job, const char *name)
+{
+ if (job == NULL)
+ return 0;
+
+ if (strcmp(job->dirname, ".glusterfs") == 0) {
+ tdbg(
+ "Directly adding directories under .glusterfs "
+ "to global list: %s\n",
+ name);
+ return 1;
+ }
+
+ if (job->parent != NULL) {
+ if (strcmp(job->parent->dirname, ".glusterfs") == 0) {
+ tdbg(
+ "Directly adding directories under .glusterfs/XX "
+ "to global list: %s\n",
+ name);
+ return 1;
+ }
+ }
+
+ return 0;
+}
+
+int
+xworker_do_crawl(struct xwork *xwork, struct dirjob *job)
+{
+ DIR *dirp = NULL;
+ int ret = -1;
+ int boff;
+ int plen;
+ char *path = NULL;
+ struct dirjob *cjob = NULL;
+ struct stat statbuf = {
+ 0,
+ };
+ struct dirent *entry;
+ struct dirent scratch[2] = {
+ {
+ 0,
+ },
+ };
+ char gfid_path[PATH_MAX] = {
+ 0,
+ };
+
+ plen = strlen(job->dirname) + 256 + 2;
+ path = alloca(plen);
+
+ tdbg("Entering: %s\n", job->dirname);
+
+ dirp = sys_opendir(job->dirname);
+ if (!dirp) {
+ terr("opendir failed on %s (%s)\n", job->dirname, strerror(errno));
+ goto out;
+ }
+
+ boff = sprintf(path, "%s/", job->dirname);
+
+ for (;;) {
+ errno = 0;
+ entry = sys_readdir(dirp, scratch);
+ if (!entry || errno != 0) {
+ if (errno != 0) {
+ err("readdir(%s): %s\n", job->dirname, strerror(errno));
+ ret = errno;
+ goto out;
+ }
+ break;
+ }
+
+ if (entry->d_ino == 0)
+ continue;
+
+ if (skip_name(job->dirname, entry->d_name))
+ continue;
+
+ /* It is sure that, children and grandchildren of .glusterfs
+ * are directories, just add them to global queue.
+ */
+ if (skip_stat(job, entry->d_name)) {
+ strncpy(path + boff, entry->d_name, (plen - boff));
+ cjob = dirjob_new(path, job);
+ if (!cjob) {
+ err("dirjob_new(%s): %s\n", path, strerror(errno));
+ ret = -1;
+ goto out;
+ }
+ xwork_addcrawl(xwork, cjob);
+ continue;
+ }
+
+ (void)snprintf(gfid_path, sizeof(gfid_path), "%s/.gfid/%s", slavemnt,
+ entry->d_name);
+ ret = sys_lstat(gfid_path, &statbuf);
+
+ if (ret && errno == ENOENT) {
+ out("%s\n", entry->d_name);
+ BUMP(skipped_gfids);
+ }
+
+ if (ret && errno != ENOENT) {
+ err("stat on slave failed(%s): %s\n", gfid_path, strerror(errno));
+ goto out;
+ }
+ }
+
+ ret = 0;
+out:
+ if (dirp)
+ (void)sys_closedir(dirp);
+
+ return ret;
+}
+
+void *
+xworker_crawl(void *data)
+{
+ struct xwork *xwork = data;
+ struct dirjob *job = NULL;
+ int ret = -1;
+
+ while ((job = xwork_pick(xwork, 0))) {
+ ret = xworker_do_crawl(xwork, job);
+ dirjob_ret(job, ret);
+ }
+
+ return NULL;
+}
+
+int
+xwork_fini(struct xwork *xwork, int stop)
+{
+ int i = 0;
+ int ret = 0;
+ void *tret = 0;
+
+ pthread_mutex_lock(&xwork->mutex);
+ {
+ xwork->stop = (xwork->stop || stop);
+ pthread_cond_broadcast(&xwork->cond);
+ }
+ pthread_mutex_unlock(&xwork->mutex);
+
+ for (i = 0; i < xwork->count; i++) {
+ pthread_join(xwork->cthreads[i], &tret);
+ tdbg("CThread id %ld returned %p\n", xwork->cthreads[i], tret);
+ }
+
+ if (debug) {
+ assert(xwork->rootjob->refcnt == 1);
+ dirjob_ret(xwork->rootjob, 0);
+ }
+
+ if (stats)
+ pthread_spin_destroy(&stats_lock);
+
+ return ret;
+}
+
+int
+xwork_init(struct xwork *xwork, int count)
+{
+ int i = 0;
+ int ret = 0;
+ struct dirjob *rootjob = NULL;
+
+ if (stats)
+ pthread_spin_init(&stats_lock, PTHREAD_PROCESS_PRIVATE);
+
+ pthread_mutex_init(&xwork->mutex, NULL);
+ pthread_cond_init(&xwork->cond, NULL);
+
+ INIT_LIST_HEAD(&xwork->crawl.list);
+
+ rootjob = dirjob_new(".glusterfs", NULL);
+ if (debug)
+ xwork->rootjob = dirjob_ref(rootjob);
+
+ xwork_addcrawl(xwork, rootjob);
+
+ xwork->count = count;
+ for (i = 0; i < count; i++) {
+ ret = pthread_create(&xwork->cthreads[i], NULL, xworker_crawl, xwork);
+ if (ret)
+ break;
+ tdbg("Spawned crawler %d thread %ld\n", i, xwork->cthreads[i]);
+ }
+
+ return ret;
+}
+
+int
+xfind(const char *basedir)
+{
+ struct xwork xwork;
+ int ret = 0;
+ char *cwd = NULL;
+
+ ret = chdir(basedir);
+ if (ret) {
+ err("%s: %s\n", basedir, strerror(errno));
+ return ret;
+ }
+
+ cwd = getcwd(0, 0);
+ if (!cwd) {
+ err("getcwd(): %s\n", strerror(errno));
+ return -1;
+ }
+
+ tdbg("Working directory: %s\n", cwd);
+ free(cwd);
+
+ memset(&xwork, 0, sizeof(xwork));
+
+ ret = xwork_init(&xwork, workers);
+ if (ret == 0)
+ xworker_crawl(&xwork);
+
+ ret = xwork_fini(&xwork, ret);
+ stats_dump();
+
+ return ret;
+}
+
+static char *
+parse_and_validate_args(int argc, char *argv[])
+{
+ char *basedir = NULL;
+ struct stat d = {
+ 0,
+ };
+ int ret = -1;
+#ifndef __FreeBSD__
+ unsigned char volume_id[16];
+#endif /* __FreeBSD__ */
+ char *slv_mnt = NULL;
+
+ if (argc != 4) {
+ err("Usage: %s <DIR> <SLAVE-VOL-MOUNT> <CRAWL-THREAD-COUNT>\n",
+ argv[0]);
+ return NULL;
+ }
+
+ basedir = argv[1];
+ ret = sys_lstat(basedir, &d);
+ if (ret) {
+ err("%s: %s\n", basedir, strerror(errno));
+ return NULL;
+ }
+
+#ifndef __FreeBSD__
+ ret = sys_lgetxattr(basedir, "trusted.glusterfs.volume-id", volume_id, 16);
+ if (ret != 16) {
+ err("%s:Not a valid brick path.\n", basedir);
+ return NULL;
+ }
+#endif /* __FreeBSD__ */
+
+ slv_mnt = argv[2];
+ ret = sys_lstat(slv_mnt, &d);
+ if (ret) {
+ err("%s: %s\n", slv_mnt, strerror(errno));
+ return NULL;
+ }
+ slavemnt = argv[2];
+
+ workers = atoi(argv[3]);
+ if (workers <= 0)
+ workers = DEFAULT_WORKERS;
+
+ return basedir;
+}
+
+int
+main(int argc, char *argv[])
+{
+ char *basedir = NULL;
+
+ basedir = parse_and_validate_args(argc, argv);
+ if (!basedir)
+ return 1;
+
+ xfind(basedir);
+
+ return 0;
+}
diff --git a/tools/gfind_missing_files/gfid_to_path.py b/tools/gfind_missing_files/gfid_to_path.py
new file mode 100644
index 00000000000..01e08a9494a
--- /dev/null
+++ b/tools/gfind_missing_files/gfid_to_path.py
@@ -0,0 +1,162 @@
+#!/usr/bin/python3
+
+# Copyright (c) 2015 Red Hat, Inc. <http://www.redhat.com/>
+# This file is part of GlusterFS.
+#
+# This file is licensed to you under your choice of the GNU Lesser
+# General Public License, version 3 or any later version (LGPLv3 or
+# later), or the GNU General Public License, version 2 (GPLv2), in all
+# cases as published by the Free Software Foundation.
+
+import sys
+import os
+import xattr
+import uuid
+import re
+import errno
+
+CHANGELOG_SEARCH_MAX_TRY = 31
+DEC_CTIME_START = 5
+ROOT_GFID = "00000000-0000-0000-0000-000000000001"
+MAX_NUM_CHANGELOGS_TRY = 2
+
+
+def output_not_found(gfid):
+ # Write GFID to stderr
+ sys.stderr.write("%s\n" % gfid)
+
+
+def output_success(path):
+ # Write converted Path to Stdout
+ sys.stdout.write("%s\n" % path)
+
+
+def full_dir_path(gfid):
+ out_path = ""
+ while True:
+ path = os.path.join(".glusterfs", gfid[0:2], gfid[2:4], gfid)
+ path_readlink = os.readlink(path)
+ pgfid = os.path.dirname(path_readlink)
+ out_path = os.path.join(os.path.basename(path_readlink), out_path)
+ if pgfid == "../../00/00/%s" % ROOT_GFID:
+ out_path = os.path.join("./", out_path)
+ break
+ gfid = os.path.basename(pgfid)
+ return out_path
+
+
+def find_path_from_changelog(fd, gfid):
+ """
+ In given Changelog File, finds using following pattern
+ <T><GFID>\x00<TYPE>\x00<MODE>\x00<UID>\x00<GID>\x00<PARGFID>/<BASENAME>
+ Pattern search finds PARGFID and BASENAME, Convert PARGFID to Path
+ Using readlink and add basename to form Full path.
+ """
+ content = fd.read()
+
+ pattern = "E%s" % gfid
+ pattern += "\x00(3|23)\x00\d+\x00\d+\x00\d+\x00([^\x00]+)/([^\x00]+)"
+ pat = re.compile(pattern)
+ match = pat.search(content)
+
+ if match:
+ pgfid = match.group(2)
+ basename = match.group(3)
+ if pgfid == ROOT_GFID:
+ return os.path.join("./", basename)
+ else:
+ full_path_parent = full_dir_path(pgfid)
+ if full_path_parent:
+ return os.path.join(full_path_parent, basename)
+
+ return None
+
+
+def gfid_to_path(gfid):
+ """
+ Try readlink, if it is directory it succeeds.
+ Get ctime of the GFID file, Decrement by 5 sec
+ Search for Changelog filename, Since Changelog file generated
+ every 15 sec, Search and get immediate next Changelog after the file
+ Creation. Get the Path by searching in Changelog file.
+ Get the resultant file's GFID and Compare with the input, If these
+ GFIDs are different then Some thing is changed(May be Rename)
+ """
+ gfid = gfid.strip()
+ gpath = os.path.join(".glusterfs", gfid[0:2], gfid[2:4], gfid)
+ try:
+ output_success(full_dir_path(gfid))
+ return
+ except OSError:
+ # Not an SymLink
+ pass
+
+ try:
+ ctime = int(os.stat(gpath).st_ctime)
+ ctime -= DEC_CTIME_START
+ except (OSError, IOError):
+ output_not_found(gfid)
+ return
+
+ path = None
+ found_changelog = False
+ changelog_parse_try = 0
+ for i in range(CHANGELOG_SEARCH_MAX_TRY):
+ cl = os.path.join(".glusterfs/changelogs", "CHANGELOG.%s" % ctime)
+
+ try:
+ with open(cl, "rb") as f:
+ changelog_parse_try += 1
+ found_changelog = True
+ path = find_path_from_changelog(f, gfid)
+ if not path and changelog_parse_try < MAX_NUM_CHANGELOGS_TRY:
+ ctime += 1
+ continue
+ break
+ except (IOError, OSError) as e:
+ if e.errno == errno.ENOENT:
+ ctime += 1
+ else:
+ break
+
+ if not found_changelog:
+ output_not_found(gfid)
+ return
+
+ if not path:
+ output_not_found(gfid)
+ return
+ gfid1 = str(uuid.UUID(bytes=xattr.get(path, "trusted.gfid")))
+ if gfid != gfid1:
+ output_not_found(gfid)
+ return
+
+ output_success(path)
+
+
+def main():
+ num_arguments = 3
+ if not sys.stdin.isatty():
+ num_arguments = 2
+
+ if len(sys.argv) != num_arguments:
+ sys.stderr.write("Invalid arguments\nUsage: "
+ "%s <BRICK_PATH> <GFID_FILE>\n" % sys.argv[0])
+ sys.exit(1)
+
+ path = sys.argv[1]
+
+ if sys.stdin.isatty():
+ gfid_list = os.path.abspath(sys.argv[2])
+ os.chdir(path)
+ with open(gfid_list) as f:
+ for gfid in f:
+ gfid_to_path(gfid)
+ else:
+ os.chdir(path)
+ for gfid in sys.stdin:
+ gfid_to_path(gfid)
+
+
+if __name__ == "__main__":
+ main()
diff --git a/tools/gfind_missing_files/gfid_to_path.sh b/tools/gfind_missing_files/gfid_to_path.sh
new file mode 100644
index 00000000000..ebe817ac2f3
--- /dev/null
+++ b/tools/gfind_missing_files/gfid_to_path.sh
@@ -0,0 +1,43 @@
+#!/bin/sh
+
+## Copyright (c) 2015 Red Hat, Inc. <http://www.redhat.com/>
+## This file is part of GlusterFS.
+##
+## This file is licensed to you under your choice of the GNU Lesser
+## General Public License, version 3 or any later version (LGPLv3 or
+## later), or the GNU General Public License, version 2 (GPLv2), in all
+## cases as published by the Free Software Foundation.
+
+E_BADARGS=65
+
+
+gfid_to_path ()
+{
+ brick_dir=$1;
+ gfid_file=$(readlink -e $2);
+
+ current_dir=$(pwd);
+ cd $brick_dir;
+
+ while read gfid
+ do
+ to_search=`echo .glusterfs/${gfid:0:2}"/"${gfid:2:2}"/"$gfid`;
+ find . -samefile $to_search | grep -v $to_search;
+ done < $gfid_file;
+
+ cd $current_dir;
+}
+
+
+main ()
+{
+ if [ $# -ne 2 ]
+ then
+ echo "Usage: `basename $0` BRICK_DIR GFID_FILE";
+ exit $E_BADARGS;
+ fi
+
+ gfid_to_path $1 $2;
+}
+
+main "$@";
diff --git a/tools/gfind_missing_files/gfind_missing_files.sh b/tools/gfind_missing_files/gfind_missing_files.sh
new file mode 100644
index 00000000000..e7aaa0b5dd4
--- /dev/null
+++ b/tools/gfind_missing_files/gfind_missing_files.sh
@@ -0,0 +1,119 @@
+#!/bin/sh
+
+## Copyright (c) 2015 Red Hat, Inc. <http://www.redhat.com/>
+## This file is part of GlusterFS.
+##
+## This file is licensed to you under your choice of the GNU Lesser
+## General Public License, version 3 or any later version (LGPLv3 or
+## later), or the GNU General Public License, version 2 (GPLv2), in all
+## cases as published by the Free Software Foundation.
+
+BRICKPATH= #Brick path of gluster volume
+SLAVEHOST= #Slave hostname
+SLAVEVOL= #Slave volume
+SLAVEMNT= #Slave gluster volume mount point
+WORKERS=4 #Default number of worker threads
+
+out()
+{
+ echo "$@";
+}
+
+fatal()
+{
+ out FATAL "$@";
+ exit 1
+}
+
+ping_host ()
+{
+ ### Use bash internal socket support
+ {
+ exec 400<>/dev/tcp/$1/$2
+ if [ $? -ne '0' ]; then
+ return 1;
+ else
+ exec 400>&-
+ return 0;
+ fi
+ } 1>&2 2>/dev/null
+}
+
+mount_slave()
+{
+ local i; # inode number
+ SSH_PORT=22
+
+ SLAVEMNT=`mktemp -d`
+ [ "x$SLAVEMNT" = "x" ] && fatal "Could not mktemp directory";
+ [ -d "$SLAVEMNT" ] || fatal "$SLAVEMNT not a directory";
+
+ ping_host ${SLAVEHOST} $SSH_PORT
+ if [ $? -ne 0 ]; then
+ echo "$SLAVEHOST not reachable.";
+ exit 1;
+ fi;
+
+ glusterfs --volfile-id=$SLAVEVOL --aux-gfid-mount --volfile-server=$SLAVEHOST $SLAVEMNT;
+ i=$(stat -c '%i' $SLAVEMNT);
+ [ "x$i" = "x1" ] || fatal "Could not mount volume $2 on $SLAVEMNT Please check host and volume exists";
+}
+
+parse_cli()
+{
+ if [ "$#" -ne 4 ]; then
+ echo "Usage: gfind_missing_files <brick-path> <slave-host> <slave-vol> <OUTFILE>"
+ exit 1
+ else
+ BRICKPATH=$1;
+ SLAVEHOST=$2;
+ SLAVEVOL=$3;
+ OUTFILE=$4;
+
+ mount_slave;
+ echo "Slave volume is mounted at ${SLAVEMNT}"
+ echo
+ fi
+}
+
+main()
+{
+ parse_cli "$@";
+
+ echo "Calling crawler...";
+ path=$(readlink -e $0)
+ $(dirname $path)/gcrawler ${BRICKPATH} ${SLAVEMNT} ${WORKERS} > ${OUTFILE}
+
+ #Clean up the mount
+ umount $SLAVEMNT;
+ rmdir $SLAVEMNT;
+
+ echo "Crawl Complete."
+ num_files_missing=$(wc -l ${OUTFILE} | awk '{print $1}')
+ if [ $num_files_missing -eq 0 ]
+ then
+ echo "Total Missing File Count : 0"
+ exit 0;
+ fi
+
+ echo "gfids of skipped files are available in the file ${OUTFILE}"
+ echo
+ echo "Starting gfid to path conversion"
+
+ #Call python script to convert gfids to full pathname
+ INFILE=$(readlink -e ${OUTFILE})
+ python $(dirname $path)/gfid_to_path.py ${BRICKPATH} ${INFILE} 1> ${OUTFILE}_pathnames 2> ${OUTFILE}_gfids
+ echo "Path names of skipped files are available in the file ${OUTFILE}_pathnames"
+
+ gfid_to_path_failures=$(wc -l ${OUTFILE}_gfids | awk '{print $1}')
+ if [ $gfid_to_path_failures -gt 0 ]
+ then
+ echo "WARNING: Unable to convert some GFIDs to Paths, GFIDs logged to ${OUTFILE}_gfids"
+ echo "Use $(dirname $path)/gfid_to_path.sh <brick-path> ${OUTFILE}_gfids to convert those GFIDs to Path"
+ fi
+
+ #Output
+ echo "Total Missing File Count : $(wc -l ${OUTFILE} | awk '{print $1}')"
+}
+
+main "$@";