@@ -79,6 +79,10 @@
org.apache.log4j
+
+ org.apache.http
+ shaded.org.apache.http
+
diff --git a/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBuffer.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBuffer.java
new file mode 100644
index 000000000..926d0b83b
--- /dev/null
+++ b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBuffer.java
@@ -0,0 +1,137 @@
+/**
+ * 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 seaweed.hdfs;
+
+import java.util.concurrent.CountDownLatch;
+
+class ReadBuffer {
+
+ private SeaweedInputStream stream;
+ private long offset; // offset within the file for the buffer
+ private int length; // actual length, set after the buffer is filles
+ private int requestedLength; // requested length of the read
+ private byte[] buffer; // the buffer itself
+ private int bufferindex = -1; // index in the buffers array in Buffer manager
+ private ReadBufferStatus status; // status of the buffer
+ private CountDownLatch latch = null; // signaled when the buffer is done reading, so any client
+ // waiting on this buffer gets unblocked
+
+ // fields to help with eviction logic
+ private long timeStamp = 0; // tick at which buffer became available to read
+ private boolean isFirstByteConsumed = false;
+ private boolean isLastByteConsumed = false;
+ private boolean isAnyByteConsumed = false;
+
+ public SeaweedInputStream getStream() {
+ return stream;
+ }
+
+ public void setStream(SeaweedInputStream stream) {
+ this.stream = stream;
+ }
+
+ public long getOffset() {
+ return offset;
+ }
+
+ public void setOffset(long offset) {
+ this.offset = offset;
+ }
+
+ public int getLength() {
+ return length;
+ }
+
+ public void setLength(int length) {
+ this.length = length;
+ }
+
+ public int getRequestedLength() {
+ return requestedLength;
+ }
+
+ public void setRequestedLength(int requestedLength) {
+ this.requestedLength = requestedLength;
+ }
+
+ public byte[] getBuffer() {
+ return buffer;
+ }
+
+ public void setBuffer(byte[] buffer) {
+ this.buffer = buffer;
+ }
+
+ public int getBufferindex() {
+ return bufferindex;
+ }
+
+ public void setBufferindex(int bufferindex) {
+ this.bufferindex = bufferindex;
+ }
+
+ public ReadBufferStatus getStatus() {
+ return status;
+ }
+
+ public void setStatus(ReadBufferStatus status) {
+ this.status = status;
+ }
+
+ public CountDownLatch getLatch() {
+ return latch;
+ }
+
+ public void setLatch(CountDownLatch latch) {
+ this.latch = latch;
+ }
+
+ public long getTimeStamp() {
+ return timeStamp;
+ }
+
+ public void setTimeStamp(long timeStamp) {
+ this.timeStamp = timeStamp;
+ }
+
+ public boolean isFirstByteConsumed() {
+ return isFirstByteConsumed;
+ }
+
+ public void setFirstByteConsumed(boolean isFirstByteConsumed) {
+ this.isFirstByteConsumed = isFirstByteConsumed;
+ }
+
+ public boolean isLastByteConsumed() {
+ return isLastByteConsumed;
+ }
+
+ public void setLastByteConsumed(boolean isLastByteConsumed) {
+ this.isLastByteConsumed = isLastByteConsumed;
+ }
+
+ public boolean isAnyByteConsumed() {
+ return isAnyByteConsumed;
+ }
+
+ public void setAnyByteConsumed(boolean isAnyByteConsumed) {
+ this.isAnyByteConsumed = isAnyByteConsumed;
+ }
+
+}
diff --git a/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferManager.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferManager.java
new file mode 100644
index 000000000..5b1e21529
--- /dev/null
+++ b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferManager.java
@@ -0,0 +1,394 @@
+/**
+ * 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 seaweed.hdfs;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.Stack;
+import java.util.concurrent.CountDownLatch;
+
+/**
+ * The Read Buffer Manager for Rest AbfsClient.
+ */
+final class ReadBufferManager {
+ private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
+
+ private static final int NUM_BUFFERS = 16;
+ private static final int BLOCK_SIZE = 4 * 1024 * 1024;
+ private static final int NUM_THREADS = 8;
+ private static final int THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold
+
+ private Thread[] threads = new Thread[NUM_THREADS];
+ private byte[][] buffers; // array of byte[] buffers, to hold the data that is read
+ private Stack freeList = new Stack<>(); // indices in buffers[] array that are available
+
+ private Queue readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
+ private LinkedList inProgressList = new LinkedList<>(); // requests being processed by worker threads
+ private LinkedList completedReadList = new LinkedList<>(); // buffers available for reading
+ private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+
+ static {
+ BUFFER_MANAGER = new ReadBufferManager();
+ BUFFER_MANAGER.init();
+ }
+
+ static ReadBufferManager getBufferManager() {
+ return BUFFER_MANAGER;
+ }
+
+ private void init() {
+ buffers = new byte[NUM_BUFFERS][];
+ for (int i = 0; i < NUM_BUFFERS; i++) {
+ buffers[i] = new byte[BLOCK_SIZE]; // same buffers are reused. The byte array never goes back to GC
+ freeList.add(i);
+ }
+ for (int i = 0; i < NUM_THREADS; i++) {
+ Thread t = new Thread(new ReadBufferWorker(i));
+ t.setDaemon(true);
+ threads[i] = t;
+ t.setName("SeaweedFS-prefetch-" + i);
+ t.start();
+ }
+ ReadBufferWorker.UNLEASH_WORKERS.countDown();
+ }
+
+ // hide instance constructor
+ private ReadBufferManager() {
+ }
+
+
+ /*
+ *
+ * SeaweedInputStream-facing methods
+ *
+ */
+
+
+ /**
+ * {@link SeaweedInputStream} calls this method to queue read-aheads.
+ *
+ * @param stream The {@link SeaweedInputStream} for which to do the read-ahead
+ * @param requestedOffset The offset in the file which shoukd be read
+ * @param requestedLength The length to read
+ */
+ void queueReadAhead(final SeaweedInputStream stream, final long requestedOffset, final int requestedLength) {
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("Start Queueing readAhead for {} offset {} length {}",
+ stream.getPath(), requestedOffset, requestedLength);
+ }
+ ReadBuffer buffer;
+ synchronized (this) {
+ if (isAlreadyQueued(stream, requestedOffset)) {
+ return; // already queued, do not queue again
+ }
+ if (freeList.isEmpty() && !tryEvict()) {
+ return; // no buffers available, cannot queue anything
+ }
+
+ buffer = new ReadBuffer();
+ buffer.setStream(stream);
+ buffer.setOffset(requestedOffset);
+ buffer.setLength(0);
+ buffer.setRequestedLength(requestedLength);
+ buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE);
+ buffer.setLatch(new CountDownLatch(1));
+
+ Integer bufferIndex = freeList.pop(); // will return a value, since we have checked size > 0 already
+
+ buffer.setBuffer(buffers[bufferIndex]);
+ buffer.setBufferindex(bufferIndex);
+ readAheadQueue.add(buffer);
+ notifyAll();
+ }
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("Done q-ing readAhead for file {} offset {} buffer idx {}",
+ stream.getPath(), requestedOffset, buffer.getBufferindex());
+ }
+ }
+
+
+ /**
+ * {@link SeaweedInputStream} calls this method read any bytes already available in a buffer (thereby saving a
+ * remote read). This returns the bytes if the data already exists in buffer. If there is a buffer that is reading
+ * the requested offset, then this method blocks until that read completes. If the data is queued in a read-ahead
+ * but not picked up by a worker thread yet, then it cancels that read-ahead and reports cache miss. This is because
+ * depending on worker thread availability, the read-ahead may take a while - the calling thread can do it's own
+ * read to get the data faster (copmared to the read waiting in queue for an indeterminate amount of time).
+ *
+ * @param stream the file to read bytes for
+ * @param position the offset in the file to do a read for
+ * @param length the length to read
+ * @param buffer the buffer to read data into. Note that the buffer will be written into from offset 0.
+ * @return the number of bytes read
+ */
+ int getBlock(final SeaweedInputStream stream, final long position, final int length, final byte[] buffer) {
+ // not synchronized, so have to be careful with locking
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("getBlock for file {} position {} thread {}",
+ stream.getPath(), position, Thread.currentThread().getName());
+ }
+
+ waitForProcess(stream, position);
+
+ int bytesRead = 0;
+ synchronized (this) {
+ bytesRead = getBlockFromCompletedQueue(stream, position, length, buffer);
+ }
+ if (bytesRead > 0) {
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("Done read from Cache for {} position {} length {}",
+ stream.getPath(), position, bytesRead);
+ }
+ return bytesRead;
+ }
+
+ // otherwise, just say we got nothing - calling thread can do its own read
+ return 0;
+ }
+
+ /*
+ *
+ * Internal methods
+ *
+ */
+
+ private void waitForProcess(final SeaweedInputStream stream, final long position) {
+ ReadBuffer readBuf;
+ synchronized (this) {
+ clearFromReadAheadQueue(stream, position);
+ readBuf = getFromList(inProgressList, stream, position);
+ }
+ if (readBuf != null) { // if in in-progress queue, then block for it
+ try {
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("got a relevant read buffer for file {} offset {} buffer idx {}",
+ stream.getPath(), readBuf.getOffset(), readBuf.getBufferindex());
+ }
+ readBuf.getLatch().await(); // blocking wait on the caller stream's thread
+ // Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread
+ // is done processing it (in doneReading). There, the latch is set after removing the buffer from
+ // inProgressList. So this latch is safe to be outside the synchronized block.
+ // Putting it in synchronized would result in a deadlock, since this thread would be holding the lock
+ // while waiting, so no one will be able to change any state. If this becomes more complex in the future,
+ // then the latch cane be removed and replaced with wait/notify whenever inProgressList is touched.
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ }
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("latch done for file {} buffer idx {} length {}",
+ stream.getPath(), readBuf.getBufferindex(), readBuf.getLength());
+ }
+ }
+ }
+
+ /**
+ * If any buffer in the completedlist can be reclaimed then reclaim it and return the buffer to free list.
+ * The objective is to find just one buffer - there is no advantage to evicting more than one.
+ *
+ * @return whether the eviction succeeeded - i.e., were we able to free up one buffer
+ */
+ private synchronized boolean tryEvict() {
+ ReadBuffer nodeToEvict = null;
+ if (completedReadList.size() <= 0) {
+ return false; // there are no evict-able buffers
+ }
+
+ // first, try buffers where all bytes have been consumed (approximated as first and last bytes consumed)
+ for (ReadBuffer buf : completedReadList) {
+ if (buf.isFirstByteConsumed() && buf.isLastByteConsumed()) {
+ nodeToEvict = buf;
+ break;
+ }
+ }
+ if (nodeToEvict != null) {
+ return evict(nodeToEvict);
+ }
+
+ // next, try buffers where any bytes have been consumed (may be a bad idea? have to experiment and see)
+ for (ReadBuffer buf : completedReadList) {
+ if (buf.isAnyByteConsumed()) {
+ nodeToEvict = buf;
+ break;
+ }
+ }
+
+ if (nodeToEvict != null) {
+ return evict(nodeToEvict);
+ }
+
+ // next, try any old nodes that have not been consumed
+ long earliestBirthday = Long.MAX_VALUE;
+ for (ReadBuffer buf : completedReadList) {
+ if (buf.getTimeStamp() < earliestBirthday) {
+ nodeToEvict = buf;
+ earliestBirthday = buf.getTimeStamp();
+ }
+ }
+ if ((currentTimeMillis() - earliestBirthday > THRESHOLD_AGE_MILLISECONDS) && (nodeToEvict != null)) {
+ return evict(nodeToEvict);
+ }
+
+ // nothing can be evicted
+ return false;
+ }
+
+ private boolean evict(final ReadBuffer buf) {
+ freeList.push(buf.getBufferindex());
+ completedReadList.remove(buf);
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
+ buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
+ }
+ return true;
+ }
+
+ private boolean isAlreadyQueued(final SeaweedInputStream stream, final long requestedOffset) {
+ // returns true if any part of the buffer is already queued
+ return (isInList(readAheadQueue, stream, requestedOffset)
+ || isInList(inProgressList, stream, requestedOffset)
+ || isInList(completedReadList, stream, requestedOffset));
+ }
+
+ private boolean isInList(final Collection list, final SeaweedInputStream stream, final long requestedOffset) {
+ return (getFromList(list, stream, requestedOffset) != null);
+ }
+
+ private ReadBuffer getFromList(final Collection list, final SeaweedInputStream stream, final long requestedOffset) {
+ for (ReadBuffer buffer : list) {
+ if (buffer.getStream() == stream) {
+ if (buffer.getStatus() == ReadBufferStatus.AVAILABLE
+ && requestedOffset >= buffer.getOffset()
+ && requestedOffset < buffer.getOffset() + buffer.getLength()) {
+ return buffer;
+ } else if (requestedOffset >= buffer.getOffset()
+ && requestedOffset < buffer.getOffset() + buffer.getRequestedLength()) {
+ return buffer;
+ }
+ }
+ }
+ return null;
+ }
+
+ private void clearFromReadAheadQueue(final SeaweedInputStream stream, final long requestedOffset) {
+ ReadBuffer buffer = getFromList(readAheadQueue, stream, requestedOffset);
+ if (buffer != null) {
+ readAheadQueue.remove(buffer);
+ notifyAll(); // lock is held in calling method
+ freeList.push(buffer.getBufferindex());
+ }
+ }
+
+ private int getBlockFromCompletedQueue(final SeaweedInputStream stream, final long position, final int length,
+ final byte[] buffer) {
+ ReadBuffer buf = getFromList(completedReadList, stream, position);
+ if (buf == null || position >= buf.getOffset() + buf.getLength()) {
+ return 0;
+ }
+ int cursor = (int) (position - buf.getOffset());
+ int availableLengthInBuffer = buf.getLength() - cursor;
+ int lengthToCopy = Math.min(length, availableLengthInBuffer);
+ System.arraycopy(buf.getBuffer(), cursor, buffer, 0, lengthToCopy);
+ if (cursor == 0) {
+ buf.setFirstByteConsumed(true);
+ }
+ if (cursor + lengthToCopy == buf.getLength()) {
+ buf.setLastByteConsumed(true);
+ }
+ buf.setAnyByteConsumed(true);
+ return lengthToCopy;
+ }
+
+ /*
+ *
+ * ReadBufferWorker-thread-facing methods
+ *
+ */
+
+ /**
+ * ReadBufferWorker thread calls this to get the next buffer that it should work on.
+ *
+ * @return {@link ReadBuffer}
+ * @throws InterruptedException if thread is interrupted
+ */
+ ReadBuffer getNextBlockToRead() throws InterruptedException {
+ ReadBuffer buffer = null;
+ synchronized (this) {
+ //buffer = readAheadQueue.take(); // blocking method
+ while (readAheadQueue.size() == 0) {
+ wait();
+ }
+ buffer = readAheadQueue.remove();
+ notifyAll();
+ if (buffer == null) {
+ return null; // should never happen
+ }
+ buffer.setStatus(ReadBufferStatus.READING_IN_PROGRESS);
+ inProgressList.add(buffer);
+ }
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("ReadBufferWorker picked file {} for offset {}",
+ buffer.getStream().getPath(), buffer.getOffset());
+ }
+ return buffer;
+ }
+
+ /**
+ * ReadBufferWorker thread calls this method to post completion.
+ *
+ * @param buffer the buffer whose read was completed
+ * @param result the {@link ReadBufferStatus} after the read operation in the worker thread
+ * @param bytesActuallyRead the number of bytes that the worker thread was actually able to read
+ */
+ void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
+ if (LOGGER.isTraceEnabled()) {
+ LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}",
+ buffer.getStream().getPath(), buffer.getOffset(), bytesActuallyRead);
+ }
+ synchronized (this) {
+ inProgressList.remove(buffer);
+ if (result == ReadBufferStatus.AVAILABLE && bytesActuallyRead > 0) {
+ buffer.setStatus(ReadBufferStatus.AVAILABLE);
+ buffer.setTimeStamp(currentTimeMillis());
+ buffer.setLength(bytesActuallyRead);
+ completedReadList.add(buffer);
+ } else {
+ freeList.push(buffer.getBufferindex());
+ // buffer should go out of scope after the end of the calling method in ReadBufferWorker, and eligible for GC
+ }
+ }
+ //outside the synchronized, since anyone receiving a wake-up from the latch must see safe-published results
+ buffer.getLatch().countDown(); // wake up waiting threads (if any)
+ }
+
+ /**
+ * Similar to System.currentTimeMillis, except implemented with System.nanoTime().
+ * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization),
+ * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing per CPU core.
+ * Note: it is not monotonic across Sockets, and even within a CPU, its only the
+ * more recent parts which share a clock across all cores.
+ *
+ * @return current time in milliseconds
+ */
+ private long currentTimeMillis() {
+ return System.nanoTime() / 1000 / 1000;
+ }
+}
diff --git a/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferStatus.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferStatus.java
new file mode 100644
index 000000000..d63674977
--- /dev/null
+++ b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferStatus.java
@@ -0,0 +1,29 @@
+/**
+ * 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 seaweed.hdfs;
+
+/**
+ * The ReadBufferStatus for Rest AbfsClient
+ */
+public enum ReadBufferStatus {
+ NOT_AVAILABLE, // buffers sitting in readaheadqueue have this stats
+ READING_IN_PROGRESS, // reading is in progress on this buffer. Buffer should be in inProgressList
+ AVAILABLE, // data is available in buffer. It should be in completedList
+ READ_FAILED // read completed, but failed.
+}
diff --git a/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferWorker.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferWorker.java
new file mode 100644
index 000000000..6ffbc4644
--- /dev/null
+++ b/other/java/hdfs3/src/main/java/seaweed/hdfs/ReadBufferWorker.java
@@ -0,0 +1,70 @@
+/**
+ * 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 seaweed.hdfs;
+
+import java.util.concurrent.CountDownLatch;
+
+class ReadBufferWorker implements Runnable {
+
+ protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1);
+ private int id;
+
+ ReadBufferWorker(final int id) {
+ this.id = id;
+ }
+
+ /**
+ * return the ID of ReadBufferWorker.
+ */
+ public int getId() {
+ return this.id;
+ }
+
+ /**
+ * Waits until a buffer becomes available in ReadAheadQueue.
+ * Once a buffer becomes available, reads the file specified in it and then posts results back to buffer manager.
+ * Rinse and repeat. Forever.
+ */
+ public void run() {
+ try {
+ UNLEASH_WORKERS.await();
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ }
+ ReadBufferManager bufferManager = ReadBufferManager.getBufferManager();
+ ReadBuffer buffer;
+ while (true) {
+ try {
+ buffer = bufferManager.getNextBlockToRead(); // blocks, until a buffer is available for this thread
+ } catch (InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ return;
+ }
+ if (buffer != null) {
+ try {
+ // do the actual read, from the file.
+ int bytesRead = buffer.getStream().readRemote(buffer.getOffset(), buffer.getBuffer(), 0, buffer.getRequestedLength());
+ bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager
+ } catch (Exception ex) {
+ bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
+ }
+ }
+ }
+ }
+}
diff --git a/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedFileSystem.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedFileSystem.java
new file mode 100644
index 000000000..c12da8261
--- /dev/null
+++ b/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedFileSystem.java
@@ -0,0 +1,620 @@
+package seaweed.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
+public class SeaweedFileSystem extends FileSystem {
+
+ public static final int FS_SEAWEED_DEFAULT_PORT = 8888;
+ public static final String FS_SEAWEED_FILER_HOST = "fs.seaweed.filer.host";
+ public static final String FS_SEAWEED_FILER_PORT = "fs.seaweed.filer.port";
+
+ private static final Logger LOG = LoggerFactory.getLogger(SeaweedFileSystem.class);
+ private static int BUFFER_SIZE = 16 * 1024 * 1024;
+
+ private URI uri;
+ private Path workingDirectory = new Path("/");
+ private SeaweedFileSystemStore seaweedFileSystemStore;
+
+ public URI getUri() {
+ return uri;
+ }
+
+ public String getScheme() {
+ return "seaweedfs";
+ }
+
+ @Override
+ public void initialize(URI uri, Configuration conf) throws IOException { // get
+ super.initialize(uri, conf);
+
+ // get host information from uri (overrides info in conf)
+ String host = uri.getHost();
+ host = (host == null) ? conf.get(FS_SEAWEED_FILER_HOST, "localhost") : host;
+ if (host == null) {
+ throw new IOException("Invalid host specified");
+ }
+ conf.set(FS_SEAWEED_FILER_HOST, host);
+
+ // get port information from uri, (overrides info in conf)
+ int port = uri.getPort();
+ port = (port == -1) ? FS_SEAWEED_DEFAULT_PORT : port;
+ conf.setInt(FS_SEAWEED_FILER_PORT, port);
+
+ conf.setInt(IO_FILE_BUFFER_SIZE_KEY, BUFFER_SIZE);
+
+ setConf(conf);
+ this.uri = uri;
+
+ seaweedFileSystemStore = new SeaweedFileSystemStore(host, port);
+
+ }
+
+ @Override
+ public FSDataInputStream open(Path path, int bufferSize) throws IOException {
+
+ LOG.debug("open path: {} bufferSize:{}", path, bufferSize);
+
+ path = qualify(path);
+
+ try {
+ InputStream inputStream = seaweedFileSystemStore.openFileForRead(path, statistics, bufferSize);
+ return new FSDataInputStream(inputStream);
+ } catch (Exception ex) {
+ LOG.warn("open path: {} bufferSize:{}", path, bufferSize, ex);
+ return null;
+ }
+ }
+
+ @Override
+ public FSDataOutputStream create(Path path, FsPermission permission, final boolean overwrite, final int bufferSize,
+ final short replication, final long blockSize, final Progressable progress) throws IOException {
+
+ LOG.debug("create path: {} bufferSize:{} blockSize:{}", path, bufferSize, blockSize);
+
+ path = qualify(path);
+
+ try {
+ String replicaPlacement = String.format("%03d", replication - 1);
+ OutputStream outputStream = seaweedFileSystemStore.createFile(path, overwrite, permission, bufferSize, replicaPlacement);
+ return new FSDataOutputStream(outputStream, statistics);
+ } catch (Exception ex) {
+ LOG.warn("create path: {} bufferSize:{} blockSize:{}", path, bufferSize, blockSize, ex);
+ return null;
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ * @throws FileNotFoundException if the parent directory is not present -or
+ * is not a directory.
+ */
+ @Override
+ public FSDataOutputStream createNonRecursive(Path path,
+ FsPermission permission,
+ EnumSet flags,
+ int bufferSize,
+ short replication,
+ long blockSize,
+ Progressable progress) throws IOException {
+ Path parent = path.getParent();
+ if (parent != null) {
+ // expect this to raise an exception if there is no parent
+ if (!getFileStatus(parent).isDirectory()) {
+ throw new FileAlreadyExistsException("Not a directory: " + parent);
+ }
+ }
+ return create(path, permission,
+ flags.contains(CreateFlag.OVERWRITE), bufferSize,
+ replication, blockSize, progress);
+ }
+
+ @Override
+ public FSDataOutputStream append(Path path, int bufferSize, Progressable progressable) throws IOException {
+
+ LOG.debug("append path: {} bufferSize:{}", path, bufferSize);
+
+ path = qualify(path);
+ try {
+ OutputStream outputStream = seaweedFileSystemStore.createFile(path, false, null, bufferSize, "");
+ return new FSDataOutputStream(outputStream, statistics);
+ } catch (Exception ex) {
+ LOG.warn("append path: {} bufferSize:{}", path, bufferSize, ex);
+ return null;
+ }
+ }
+
+ @Override
+ public boolean rename(Path src, Path dst) {
+
+ LOG.debug("rename path: {} => {}", src, dst);
+
+ if (src.isRoot()) {
+ return false;
+ }
+
+ if (src.equals(dst)) {
+ return true;
+ }
+ FileStatus dstFileStatus = getFileStatus(dst);
+
+ String sourceFileName = src.getName();
+ Path adjustedDst = dst;
+
+ if (dstFileStatus != null) {
+ if (!dstFileStatus.isDirectory()) {
+ return false;
+ }
+ adjustedDst = new Path(dst, sourceFileName);
+ }
+
+ Path qualifiedSrcPath = qualify(src);
+ Path qualifiedDstPath = qualify(adjustedDst);
+
+ seaweedFileSystemStore.rename(qualifiedSrcPath, qualifiedDstPath);
+ return true;
+ }
+
+ @Override
+ public boolean delete(Path path, boolean recursive) {
+
+ LOG.debug("delete path: {} recursive:{}", path, recursive);
+
+ path = qualify(path);
+
+ FileStatus fileStatus = getFileStatus(path);
+
+ if (fileStatus == null) {
+ return true;
+ }
+
+ return seaweedFileSystemStore.deleteEntries(path, fileStatus.isDirectory(), recursive);
+
+ }
+
+ @Override
+ public FileStatus[] listStatus(Path path) throws IOException {
+
+ LOG.debug("listStatus path: {}", path);
+
+ path = qualify(path);
+
+ return seaweedFileSystemStore.listEntries(path);
+ }
+
+ @Override
+ public Path getWorkingDirectory() {
+ return workingDirectory;
+ }
+
+ @Override
+ public void setWorkingDirectory(Path path) {
+ if (path.isAbsolute()) {
+ workingDirectory = path;
+ } else {
+ workingDirectory = new Path(workingDirectory, path);
+ }
+ }
+
+ @Override
+ public boolean mkdirs(Path path, FsPermission fsPermission) throws IOException {
+
+ LOG.debug("mkdirs path: {}", path);
+
+ path = qualify(path);
+
+ FileStatus fileStatus = getFileStatus(path);
+
+ if (fileStatus == null) {
+
+ UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+ return seaweedFileSystemStore.createDirectory(path, currentUser,
+ fsPermission == null ? FsPermission.getDirDefault() : fsPermission,
+ FsPermission.getUMask(getConf()));
+
+ }
+
+ if (fileStatus.isDirectory()) {
+ return true;
+ } else {
+ throw new FileAlreadyExistsException("Path is a file: " + path);
+ }
+ }
+
+ @Override
+ public FileStatus getFileStatus(Path path) {
+
+ LOG.debug("getFileStatus path: {}", path);
+
+ path = qualify(path);
+
+ return seaweedFileSystemStore.getFileStatus(path);
+ }
+
+ /**
+ * Set owner of a path (i.e. a file or a directory).
+ * The parameters owner and group cannot both be null.
+ *
+ * @param path The path
+ * @param owner If it is null, the original username remains unchanged.
+ * @param group If it is null, the original groupname remains unchanged.
+ */
+ @Override
+ public void setOwner(Path path, final String owner, final String group)
+ throws IOException {
+ LOG.debug("setOwner path: {}", path);
+ path = qualify(path);
+
+ seaweedFileSystemStore.setOwner(path, owner, group);
+ }
+
+
+ /**
+ * Set permission of a path.
+ *
+ * @param path The path
+ * @param permission Access permission
+ */
+ @Override
+ public void setPermission(Path path, final FsPermission permission) throws IOException {
+ LOG.debug("setPermission path: {}", path);
+
+ if (permission == null) {
+ throw new IllegalArgumentException("The permission can't be null");
+ }
+
+ path = qualify(path);
+
+ seaweedFileSystemStore.setPermission(path, permission);
+ }
+
+ Path qualify(Path path) {
+ return path.makeQualified(uri, workingDirectory);
+ }
+
+ /**
+ * Concat existing files together.
+ *
+ * @param trg the path to the target destination.
+ * @param psrcs the paths to the sources to use for the concatenation.
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default).
+ */
+ @Override
+ public void concat(final Path trg, final Path[] psrcs) throws IOException {
+ throw new UnsupportedOperationException("Not implemented by the " +
+ getClass().getSimpleName() + " FileSystem implementation");
+ }
+
+ /**
+ * Truncate the file in the indicated path to the indicated size.
+ *
+ * Fails if path is a directory.
+ * Fails if path does not exist.
+ * Fails if path is not closed.
+ * Fails if new size is greater than current size.
+ *
+ *
+ * @param f The path to the file to be truncated
+ * @param newLength The size the file is to be truncated to
+ * @return true
if the file has been truncated to the desired
+ * newLength
and is immediately available to be reused for
+ * write operations such as append
, or
+ * false
if a background process of adjusting the length of
+ * the last block has been started, and clients should wait for it to
+ * complete before proceeding with further file updates.
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default).
+ */
+ @Override
+ public boolean truncate(Path f, long newLength) throws IOException {
+ throw new UnsupportedOperationException("Not implemented by the " +
+ getClass().getSimpleName() + " FileSystem implementation");
+ }
+
+ @Override
+ public void createSymlink(final Path target, final Path link,
+ final boolean createParent) throws AccessControlException,
+ FileAlreadyExistsException, FileNotFoundException,
+ ParentNotDirectoryException, UnsupportedFileSystemException,
+ IOException {
+ // Supporting filesystems should override this method
+ throw new UnsupportedOperationException(
+ "Filesystem does not support symlinks!");
+ }
+
+ public boolean supportsSymlinks() {
+ return false;
+ }
+
+ /**
+ * Create a snapshot.
+ *
+ * @param path The directory where snapshots will be taken.
+ * @param snapshotName The name of the snapshot
+ * @return the snapshot path.
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ */
+ @Override
+ public Path createSnapshot(Path path, String snapshotName)
+ throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support createSnapshot");
+ }
+
+ /**
+ * Rename a snapshot.
+ *
+ * @param path The directory path where the snapshot was taken
+ * @param snapshotOldName Old name of the snapshot
+ * @param snapshotNewName New name of the snapshot
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void renameSnapshot(Path path, String snapshotOldName,
+ String snapshotNewName) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support renameSnapshot");
+ }
+
+ /**
+ * Delete a snapshot of a directory.
+ *
+ * @param path The directory that the to-be-deleted snapshot belongs to
+ * @param snapshotName The name of the snapshot
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void deleteSnapshot(Path path, String snapshotName)
+ throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support deleteSnapshot");
+ }
+
+ /**
+ * Modifies ACL entries of files and directories. This method can add new ACL
+ * entries or modify the permissions on existing ACL entries. All existing
+ * ACL entries that are not specified in this call are retained without
+ * changes. (Modifications are merged into the current ACL.)
+ *
+ * @param path Path to modify
+ * @param aclSpec List<AclEntry> describing modifications
+ * @throws IOException if an ACL could not be modified
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void modifyAclEntries(Path path, List aclSpec)
+ throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support modifyAclEntries");
+ }
+
+ /**
+ * Removes ACL entries from files and directories. Other ACL entries are
+ * retained.
+ *
+ * @param path Path to modify
+ * @param aclSpec List describing entries to remove
+ * @throws IOException if an ACL could not be modified
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void removeAclEntries(Path path, List aclSpec)
+ throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support removeAclEntries");
+ }
+
+ /**
+ * Removes all default ACL entries from files and directories.
+ *
+ * @param path Path to modify
+ * @throws IOException if an ACL could not be modified
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void removeDefaultAcl(Path path)
+ throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support removeDefaultAcl");
+ }
+
+ /**
+ * Removes all but the base ACL entries of files and directories. The entries
+ * for user, group, and others are retained for compatibility with permission
+ * bits.
+ *
+ * @param path Path to modify
+ * @throws IOException if an ACL could not be removed
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void removeAcl(Path path)
+ throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support removeAcl");
+ }
+
+ /**
+ * Fully replaces ACL of files and directories, discarding all existing
+ * entries.
+ *
+ * @param path Path to modify
+ * @param aclSpec List describing modifications, which must include entries
+ * for user, group, and others for compatibility with permission bits.
+ * @throws IOException if an ACL could not be modified
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void setAcl(Path path, List aclSpec) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support setAcl");
+ }
+
+ /**
+ * Gets the ACL of a file or directory.
+ *
+ * @param path Path to get
+ * @return AclStatus describing the ACL of the file or directory
+ * @throws IOException if an ACL could not be read
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public AclStatus getAclStatus(Path path) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support getAclStatus");
+ }
+
+ /**
+ * Set an xattr of a file or directory.
+ * The name must be prefixed with the namespace followed by ".". For example,
+ * "user.attr".
+ *
+ * Refer to the HDFS extended attributes user documentation for details.
+ *
+ * @param path Path to modify
+ * @param name xattr name.
+ * @param value xattr value.
+ * @param flag xattr set flag
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void setXAttr(Path path, String name, byte[] value,
+ EnumSet flag) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support setXAttr");
+ }
+
+ /**
+ * Get an xattr name and value for a file or directory.
+ * The name must be prefixed with the namespace followed by ".". For example,
+ * "user.attr".
+ *
+ * Refer to the HDFS extended attributes user documentation for details.
+ *
+ * @param path Path to get extended attribute
+ * @param name xattr name.
+ * @return byte[] xattr value.
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public byte[] getXAttr(Path path, String name) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support getXAttr");
+ }
+
+ /**
+ * Get all of the xattr name/value pairs for a file or directory.
+ * Only those xattrs which the logged-in user has permissions to view
+ * are returned.
+ *
+ * Refer to the HDFS extended attributes user documentation for details.
+ *
+ * @param path Path to get extended attributes
+ * @return Map describing the XAttrs of the file or directory
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public Map getXAttrs(Path path) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support getXAttrs");
+ }
+
+ /**
+ * Get all of the xattrs name/value pairs for a file or directory.
+ * Only those xattrs which the logged-in user has permissions to view
+ * are returned.
+ *
+ * Refer to the HDFS extended attributes user documentation for details.
+ *
+ * @param path Path to get extended attributes
+ * @param names XAttr names.
+ * @return Map describing the XAttrs of the file or directory
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public Map getXAttrs(Path path, List names)
+ throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support getXAttrs");
+ }
+
+ /**
+ * Get all of the xattr names for a file or directory.
+ * Only those xattr names which the logged-in user has permissions to view
+ * are returned.
+ *
+ * Refer to the HDFS extended attributes user documentation for details.
+ *
+ * @param path Path to get extended attributes
+ * @return List{@literal } of the XAttr names of the file or directory
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public List listXAttrs(Path path) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support listXAttrs");
+ }
+
+ /**
+ * Remove an xattr of a file or directory.
+ * The name must be prefixed with the namespace followed by ".". For example,
+ * "user.attr".
+ *
+ * Refer to the HDFS extended attributes user documentation for details.
+ *
+ * @param path Path to remove extended attribute
+ * @param name xattr name
+ * @throws IOException IO failure
+ * @throws UnsupportedOperationException if the operation is unsupported
+ * (default outcome).
+ */
+ @Override
+ public void removeXAttr(Path path, String name) throws IOException {
+ throw new UnsupportedOperationException(getClass().getSimpleName()
+ + " doesn't support removeXAttr");
+ }
+
+}
diff --git a/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedFileSystemStore.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedFileSystemStore.java
new file mode 100644
index 000000000..774c090e8
--- /dev/null
+++ b/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedFileSystemStore.java
@@ -0,0 +1,270 @@
+package seaweed.hdfs;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import seaweedfs.client.FilerClient;
+import seaweedfs.client.FilerGrpcClient;
+import seaweedfs.client.FilerProto;
+import seaweedfs.client.SeaweedRead;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class SeaweedFileSystemStore {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SeaweedFileSystemStore.class);
+
+ private FilerGrpcClient filerGrpcClient;
+ private FilerClient filerClient;
+
+ public SeaweedFileSystemStore(String host, int port) {
+ int grpcPort = 10000 + port;
+ filerGrpcClient = new FilerGrpcClient(host, grpcPort);
+ filerClient = new FilerClient(filerGrpcClient);
+ }
+
+ public static String getParentDirectory(Path path) {
+ return path.isRoot() ? "/" : path.getParent().toUri().getPath();
+ }
+
+ static int permissionToMode(FsPermission permission, boolean isDirectory) {
+ int p = permission.toShort();
+ if (isDirectory) {
+ p = p | 1 << 31;
+ }
+ return p;
+ }
+
+ public boolean createDirectory(final Path path, UserGroupInformation currentUser,
+ final FsPermission permission, final FsPermission umask) {
+
+ LOG.debug("createDirectory path: {} permission: {} umask: {}",
+ path,
+ permission,
+ umask);
+
+ return filerClient.mkdirs(
+ path.toUri().getPath(),
+ permissionToMode(permission, true),
+ currentUser.getUserName(),
+ currentUser.getGroupNames()
+ );
+ }
+
+ public FileStatus[] listEntries(final Path path) {
+ LOG.debug("listEntries path: {}", path);
+
+ List fileStatuses = new ArrayList();
+
+ List entries = filerClient.listEntries(path.toUri().getPath());
+
+ for (FilerProto.Entry entry : entries) {
+
+ FileStatus fileStatus = doGetFileStatus(new Path(path, entry.getName()), entry);
+
+ fileStatuses.add(fileStatus);
+ }
+ return fileStatuses.toArray(new FileStatus[0]);
+ }
+
+ public FileStatus getFileStatus(final Path path) {
+
+ FilerProto.Entry entry = lookupEntry(path);
+ if (entry == null) {
+ return null;
+ }
+ LOG.debug("doGetFileStatus path:{} entry:{}", path, entry);
+
+ FileStatus fileStatus = doGetFileStatus(path, entry);
+ return fileStatus;
+ }
+
+ public boolean deleteEntries(final Path path, boolean isDirectory, boolean recursive) {
+ LOG.debug("deleteEntries path: {} isDirectory {} recursive: {}",
+ path,
+ String.valueOf(isDirectory),
+ String.valueOf(recursive));
+
+ if (path.isRoot()) {
+ return true;
+ }
+
+ if (recursive && isDirectory) {
+ List entries = filerClient.listEntries(path.toUri().getPath());
+ for (FilerProto.Entry entry : entries) {
+ deleteEntries(new Path(path, entry.getName()), entry.getIsDirectory(), true);
+ }
+ }
+
+ return filerClient.deleteEntry(getParentDirectory(path), path.getName(), true, recursive, true);
+ }
+
+ private FileStatus doGetFileStatus(Path path, FilerProto.Entry entry) {
+ FilerProto.FuseAttributes attributes = entry.getAttributes();
+ long length = SeaweedRead.totalSize(entry.getChunksList());
+ boolean isDir = entry.getIsDirectory();
+ int block_replication = 1;
+ int blocksize = 512;
+ long modification_time = attributes.getMtime() * 1000; // milliseconds
+ long access_time = 0;
+ FsPermission permission = FsPermission.createImmutable((short) attributes.getFileMode());
+ String owner = attributes.getUserName();
+ String group = attributes.getGroupNameCount() > 0 ? attributes.getGroupName(0) : "";
+ return new FileStatus(length, isDir, block_replication, blocksize,
+ modification_time, access_time, permission, owner, group, null, path);
+ }
+
+ private FilerProto.Entry lookupEntry(Path path) {
+
+ return filerClient.lookupEntry(getParentDirectory(path), path.getName());
+
+ }
+
+ public void rename(Path source, Path destination) {
+
+ LOG.debug("rename source: {} destination:{}", source, destination);
+
+ if (source.isRoot()) {
+ return;
+ }
+ LOG.info("rename source: {} destination:{}", source, destination);
+ FilerProto.Entry entry = lookupEntry(source);
+ if (entry == null) {
+ LOG.warn("rename non-existing source: {}", source);
+ return;
+ }
+ filerClient.mv(source.toUri().getPath(), destination.toUri().getPath());
+ }
+
+ public OutputStream createFile(final Path path,
+ final boolean overwrite,
+ FsPermission permission,
+ int bufferSize,
+ String replication) throws IOException {
+
+ permission = permission == null ? FsPermission.getFileDefault() : permission;
+
+ LOG.debug("createFile path: {} overwrite: {} permission: {}",
+ path,
+ overwrite,
+ permission.toString());
+
+ UserGroupInformation userGroupInformation = UserGroupInformation.getCurrentUser();
+ long now = System.currentTimeMillis() / 1000L;
+
+ FilerProto.Entry.Builder entry = null;
+ long writePosition = 0;
+ if (!overwrite) {
+ FilerProto.Entry existingEntry = lookupEntry(path);
+ LOG.debug("createFile merged entry path:{} existingEntry:{}", path, existingEntry);
+ if (existingEntry != null) {
+ entry = FilerProto.Entry.newBuilder();
+ entry.mergeFrom(existingEntry);
+ entry.getAttributesBuilder().setMtime(now);
+ LOG.debug("createFile merged entry path:{} entry:{} from:{}", path, entry, existingEntry);
+ writePosition = SeaweedRead.totalSize(existingEntry.getChunksList());
+ replication = existingEntry.getAttributes().getReplication();
+ }
+ }
+ if (entry == null) {
+ entry = FilerProto.Entry.newBuilder()
+ .setName(path.getName())
+ .setIsDirectory(false)
+ .setAttributes(FilerProto.FuseAttributes.newBuilder()
+ .setFileMode(permissionToMode(permission, false))
+ .setReplication(replication)
+ .setCrtime(now)
+ .setMtime(now)
+ .setUserName(userGroupInformation.getUserName())
+ .clearGroupName()
+ .addAllGroupName(Arrays.asList(userGroupInformation.getGroupNames()))
+ );
+ }
+
+ return new SeaweedOutputStream(filerGrpcClient, path, entry, writePosition, bufferSize, replication);
+
+ }
+
+ public InputStream openFileForRead(final Path path, FileSystem.Statistics statistics,
+ int bufferSize) throws IOException {
+
+ LOG.debug("openFileForRead path:{} bufferSize:{}", path, bufferSize);
+
+ int readAheadQueueDepth = 2;
+ FilerProto.Entry entry = lookupEntry(path);
+
+ if (entry == null) {
+ throw new FileNotFoundException("read non-exist file " + path);
+ }
+
+ return new SeaweedInputStream(filerGrpcClient,
+ statistics,
+ path.toUri().getPath(),
+ entry,
+ bufferSize,
+ readAheadQueueDepth);
+ }
+
+ public void setOwner(Path path, String owner, String group) {
+
+ LOG.debug("setOwner path:{} owner:{} group:{}", path, owner, group);
+
+ FilerProto.Entry entry = lookupEntry(path);
+ if (entry == null) {
+ LOG.debug("setOwner path:{} entry:{}", path, entry);
+ return;
+ }
+
+ FilerProto.Entry.Builder entryBuilder = entry.toBuilder();
+ FilerProto.FuseAttributes.Builder attributesBuilder = entry.getAttributes().toBuilder();
+
+ if (owner != null) {
+ attributesBuilder.setUserName(owner);
+ }
+ if (group != null) {
+ attributesBuilder.clearGroupName();
+ attributesBuilder.addGroupName(group);
+ }
+
+ entryBuilder.setAttributes(attributesBuilder);
+
+ LOG.debug("setOwner path:{} entry:{}", path, entryBuilder);
+
+ filerClient.updateEntry(getParentDirectory(path), entryBuilder.build());
+
+ }
+
+ public void setPermission(Path path, FsPermission permission) {
+
+ LOG.debug("setPermission path:{} permission:{}", path, permission);
+
+ FilerProto.Entry entry = lookupEntry(path);
+ if (entry == null) {
+ LOG.debug("setPermission path:{} entry:{}", path, entry);
+ return;
+ }
+
+ FilerProto.Entry.Builder entryBuilder = entry.toBuilder();
+ FilerProto.FuseAttributes.Builder attributesBuilder = entry.getAttributes().toBuilder();
+
+ attributesBuilder.setFileMode(permissionToMode(permission, entry.getIsDirectory()));
+
+ entryBuilder.setAttributes(attributesBuilder);
+
+ LOG.debug("setPermission path:{} entry:{}", path, entryBuilder);
+
+ filerClient.updateEntry(getParentDirectory(path), entryBuilder.build());
+
+ }
+
+}
diff --git a/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedInputStream.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedInputStream.java
new file mode 100644
index 000000000..90c14c772
--- /dev/null
+++ b/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedInputStream.java
@@ -0,0 +1,371 @@
+package seaweed.hdfs;
+
+// based on org.apache.hadoop.fs.azurebfs.services.AbfsInputStream
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import seaweedfs.client.FilerGrpcClient;
+import seaweedfs.client.FilerProto;
+import seaweedfs.client.SeaweedRead;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.List;
+
+public class SeaweedInputStream extends FSInputStream {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SeaweedInputStream.class);
+
+ private final FilerGrpcClient filerGrpcClient;
+ private final Statistics statistics;
+ private final String path;
+ private final FilerProto.Entry entry;
+ private final List visibleIntervalList;
+ private final long contentLength;
+ private final int bufferSize; // default buffer size
+ private final int readAheadQueueDepth; // initialized in constructor
+ private final boolean readAheadEnabled; // whether enable readAhead;
+
+ private byte[] buffer = null; // will be initialized on first use
+
+ private long fCursor = 0; // cursor of buffer within file - offset of next byte to read from remote server
+ private long fCursorAfterLastRead = -1;
+ private int bCursor = 0; // cursor of read within buffer - offset of next byte to be returned from buffer
+ private int limit = 0; // offset of next byte to be read into buffer from service (i.e., upper marker+1
+ // of valid bytes in buffer)
+ private boolean closed = false;
+
+ public SeaweedInputStream(
+ final FilerGrpcClient filerGrpcClient,
+ final Statistics statistics,
+ final String path,
+ final FilerProto.Entry entry,
+ final int bufferSize,
+ final int readAheadQueueDepth) {
+ this.filerGrpcClient = filerGrpcClient;
+ this.statistics = statistics;
+ this.path = path;
+ this.entry = entry;
+ this.contentLength = SeaweedRead.totalSize(entry.getChunksList());
+ this.bufferSize = bufferSize;
+ this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : Runtime.getRuntime().availableProcessors();
+ this.readAheadEnabled = true;
+
+ this.visibleIntervalList = SeaweedRead.nonOverlappingVisibleIntervals(entry.getChunksList());
+
+ LOG.debug("new path:{} entry:{} visibleIntervalList:{}", path, entry, visibleIntervalList);
+
+ }
+
+ public String getPath() {
+ return path;
+ }
+
+ @Override
+ public int read() throws IOException {
+ byte[] b = new byte[1];
+ int numberOfBytesRead = read(b, 0, 1);
+ if (numberOfBytesRead < 0) {
+ return -1;
+ } else {
+ return (b[0] & 0xFF);
+ }
+ }
+
+ @Override
+ public synchronized int read(final byte[] b, final int off, final int len) throws IOException {
+ int currentOff = off;
+ int currentLen = len;
+ int lastReadBytes;
+ int totalReadBytes = 0;
+ do {
+ lastReadBytes = readOneBlock(b, currentOff, currentLen);
+ if (lastReadBytes > 0) {
+ currentOff += lastReadBytes;
+ currentLen -= lastReadBytes;
+ totalReadBytes += lastReadBytes;
+ }
+ if (currentLen <= 0 || currentLen > b.length - currentOff) {
+ break;
+ }
+ } while (lastReadBytes > 0);
+ return totalReadBytes > 0 ? totalReadBytes : lastReadBytes;
+ }
+
+ private int readOneBlock(final byte[] b, final int off, final int len) throws IOException {
+ if (closed) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+
+ Preconditions.checkNotNull(b);
+
+ if (len == 0) {
+ return 0;
+ }
+
+ if (this.available() == 0) {
+ return -1;
+ }
+
+ if (off < 0 || len < 0 || len > b.length - off) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ //If buffer is empty, then fill the buffer.
+ if (bCursor == limit) {
+ //If EOF, then return -1
+ if (fCursor >= contentLength) {
+ return -1;
+ }
+
+ long bytesRead = 0;
+ //reset buffer to initial state - i.e., throw away existing data
+ bCursor = 0;
+ limit = 0;
+ if (buffer == null) {
+ buffer = new byte[bufferSize];
+ }
+
+ // Enable readAhead when reading sequentially
+ if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
+ bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);
+ } else {
+ bytesRead = readInternal(fCursor, buffer, 0, b.length, true);
+ }
+
+ if (bytesRead == -1) {
+ return -1;
+ }
+
+ limit += bytesRead;
+ fCursor += bytesRead;
+ fCursorAfterLastRead = fCursor;
+ }
+
+ //If there is anything in the buffer, then return lesser of (requested bytes) and (bytes in buffer)
+ //(bytes returned may be less than requested)
+ int bytesRemaining = limit - bCursor;
+ int bytesToRead = Math.min(len, bytesRemaining);
+ System.arraycopy(buffer, bCursor, b, off, bytesToRead);
+ bCursor += bytesToRead;
+ if (statistics != null) {
+ statistics.incrementBytesRead(bytesToRead);
+ }
+ return bytesToRead;
+ }
+
+
+ private int readInternal(final long position, final byte[] b, final int offset, final int length,
+ final boolean bypassReadAhead) throws IOException {
+ if (readAheadEnabled && !bypassReadAhead) {
+ // try reading from read-ahead
+ if (offset != 0) {
+ throw new IllegalArgumentException("readahead buffers cannot have non-zero buffer offsets");
+ }
+ int receivedBytes;
+
+ // queue read-aheads
+ int numReadAheads = this.readAheadQueueDepth;
+ long nextSize;
+ long nextOffset = position;
+ while (numReadAheads > 0 && nextOffset < contentLength) {
+ nextSize = Math.min((long) bufferSize, contentLength - nextOffset);
+ ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize);
+ nextOffset = nextOffset + nextSize;
+ numReadAheads--;
+ }
+
+ // try reading from buffers first
+ receivedBytes = ReadBufferManager.getBufferManager().getBlock(this, position, length, b);
+ if (receivedBytes > 0) {
+ return receivedBytes;
+ }
+
+ // got nothing from read-ahead, do our own read now
+ receivedBytes = readRemote(position, b, offset, length);
+ return receivedBytes;
+ } else {
+ return readRemote(position, b, offset, length);
+ }
+ }
+
+ int readRemote(long position, byte[] b, int offset, int length) throws IOException {
+ if (position < 0) {
+ throw new IllegalArgumentException("attempting to read from negative offset");
+ }
+ if (position >= contentLength) {
+ return -1; // Hadoop prefers -1 to EOFException
+ }
+ if (b == null) {
+ throw new IllegalArgumentException("null byte array passed in to read() method");
+ }
+ if (offset >= b.length) {
+ throw new IllegalArgumentException("offset greater than length of array");
+ }
+ if (length < 0) {
+ throw new IllegalArgumentException("requested read length is less than zero");
+ }
+ if (length > (b.length - offset)) {
+ throw new IllegalArgumentException("requested read length is more than will fit after requested offset in buffer");
+ }
+
+ long bytesRead = SeaweedRead.read(filerGrpcClient, visibleIntervalList, position, b, offset, length);
+ if (bytesRead > Integer.MAX_VALUE) {
+ throw new IOException("Unexpected Content-Length");
+ }
+ return (int) bytesRead;
+ }
+
+ /**
+ * Seek to given position in stream.
+ *
+ * @param n position to seek to
+ * @throws IOException if there is an error
+ * @throws EOFException if attempting to seek past end of file
+ */
+ @Override
+ public synchronized void seek(long n) throws IOException {
+ if (closed) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ if (n < 0) {
+ throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+ }
+ if (n > contentLength) {
+ throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+ }
+
+ if (n >= fCursor - limit && n <= fCursor) { // within buffer
+ bCursor = (int) (n - (fCursor - limit));
+ return;
+ }
+
+ // next read will read from here
+ fCursor = n;
+
+ //invalidate buffer
+ limit = 0;
+ bCursor = 0;
+ }
+
+ @Override
+ public synchronized long skip(long n) throws IOException {
+ if (closed) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ long currentPos = getPos();
+ if (currentPos == contentLength) {
+ if (n > 0) {
+ throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+ }
+ }
+ long newPos = currentPos + n;
+ if (newPos < 0) {
+ newPos = 0;
+ n = newPos - currentPos;
+ }
+ if (newPos > contentLength) {
+ newPos = contentLength;
+ n = newPos - currentPos;
+ }
+ seek(newPos);
+ return n;
+ }
+
+ /**
+ * Return the size of the remaining available bytes
+ * if the size is less than or equal to {@link Integer#MAX_VALUE},
+ * otherwise, return {@link Integer#MAX_VALUE}.
+ *
+ * This is to match the behavior of DFSInputStream.available(),
+ * which some clients may rely on (HBase write-ahead log reading in
+ * particular).
+ */
+ @Override
+ public synchronized int available() throws IOException {
+ if (closed) {
+ throw new IOException(
+ FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ final long remaining = this.contentLength - this.getPos();
+ return remaining <= Integer.MAX_VALUE
+ ? (int) remaining : Integer.MAX_VALUE;
+ }
+
+ /**
+ * Returns the length of the file that this stream refers to. Note that the length returned is the length
+ * as of the time the Stream was opened. Specifically, if there have been subsequent appends to the file,
+ * they wont be reflected in the returned length.
+ *
+ * @return length of the file.
+ * @throws IOException if the stream is closed
+ */
+ public long length() throws IOException {
+ if (closed) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ return contentLength;
+ }
+
+ /**
+ * Return the current offset from the start of the file
+ *
+ * @throws IOException throws {@link IOException} if there is an error
+ */
+ @Override
+ public synchronized long getPos() throws IOException {
+ if (closed) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ return fCursor - limit + bCursor;
+ }
+
+ /**
+ * Seeks a different copy of the data. Returns true if
+ * found a new source, false otherwise.
+ *
+ * @throws IOException throws {@link IOException} if there is an error
+ */
+ @Override
+ public boolean seekToNewSource(long l) throws IOException {
+ return false;
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
+ closed = true;
+ buffer = null; // de-reference the buffer so it can be GC'ed sooner
+ }
+
+ /**
+ * Not supported by this stream. Throws {@link UnsupportedOperationException}
+ *
+ * @param readlimit ignored
+ */
+ @Override
+ public synchronized void mark(int readlimit) {
+ throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
+ }
+
+ /**
+ * Not supported by this stream. Throws {@link UnsupportedOperationException}
+ */
+ @Override
+ public synchronized void reset() throws IOException {
+ throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
+ }
+
+ /**
+ * gets whether mark and reset are supported by {@code ADLFileInputStream}. Always returns false.
+ *
+ * @return always {@code false}
+ */
+ @Override
+ public boolean markSupported() {
+ return false;
+ }
+}
diff --git a/other/java/hdfs/src/main/java/seaweed/hdfs/SeaweedOutputStream.java b/other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedOutputStream.java
similarity index 100%
rename from other/java/hdfs/src/main/java/seaweed/hdfs/SeaweedOutputStream.java
rename to other/java/hdfs3/src/main/java/seaweed/hdfs/SeaweedOutputStream.java
diff --git a/unmaintained/change_superblock/change_superblock.go b/unmaintained/change_superblock/change_superblock.go
index 07d9b94e4..afe651c4e 100644
--- a/unmaintained/change_superblock/change_superblock.go
+++ b/unmaintained/change_superblock/change_superblock.go
@@ -8,8 +8,9 @@ import (
"strconv"
"github.com/chrislusf/seaweedfs/weed/glog"
- "github.com/chrislusf/seaweedfs/weed/storage"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
var (
@@ -47,9 +48,10 @@ func main() {
if err != nil {
glog.Fatalf("Open Volume Data File [ERROR]: %v", err)
}
- defer datFile.Close()
+ datBackend := backend.NewDiskFile(datFile)
+ defer datBackend.Close()
- superBlock, err := storage.ReadSuperBlock(datFile)
+ superBlock, err := super_block.ReadSuperBlock(datBackend)
if err != nil {
glog.Fatalf("cannot parse existing super block: %v", err)
@@ -61,7 +63,7 @@ func main() {
hasChange := false
if *targetReplica != "" {
- replica, err := storage.NewReplicaPlacementFromString(*targetReplica)
+ replica, err := super_block.NewReplicaPlacementFromString(*targetReplica)
if err != nil {
glog.Fatalf("cannot parse target replica %s: %v", *targetReplica, err)
diff --git a/unmaintained/fix_dat/fix_dat.go b/unmaintained/fix_dat/fix_dat.go
index a72a78eed..d6110d870 100644
--- a/unmaintained/fix_dat/fix_dat.go
+++ b/unmaintained/fix_dat/fix_dat.go
@@ -9,8 +9,9 @@ import (
"strconv"
"github.com/chrislusf/seaweedfs/weed/glog"
- "github.com/chrislusf/seaweedfs/weed/storage"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -44,11 +45,13 @@ func main() {
glog.Fatalf("Read Volume Index %v", err)
}
defer indexFile.Close()
- datFile, err := os.OpenFile(path.Join(*fixVolumePath, fileName+".dat"), os.O_RDONLY, 0644)
+ datFileName := path.Join(*fixVolumePath, fileName+".dat")
+ datFile, err := os.OpenFile(datFileName, os.O_RDONLY, 0644)
if err != nil {
glog.Fatalf("Read Volume Data %v", err)
}
- defer datFile.Close()
+ datBackend := backend.NewDiskFile(datFile)
+ defer datBackend.Close()
newDatFile, err := os.Create(path.Join(*fixVolumePath, fileName+".dat_fixed"))
if err != nil {
@@ -56,21 +59,21 @@ func main() {
}
defer newDatFile.Close()
- superBlock, err := storage.ReadSuperBlock(datFile)
+ superBlock, err := super_block.ReadSuperBlock(datBackend)
if err != nil {
glog.Fatalf("Read Volume Data superblock %v", err)
}
newDatFile.Write(superBlock.Bytes())
- iterateEntries(datFile, indexFile, func(n *needle.Needle, offset int64) {
+ iterateEntries(datBackend, indexFile, func(n *needle.Needle, offset int64) {
fmt.Printf("needle id=%v name=%s size=%d dataSize=%d\n", n.Id, string(n.Name), n.Size, n.DataSize)
- _, s, _, e := n.Append(newDatFile, superBlock.Version())
+ _, s, _, e := n.Append(datBackend, superBlock.Version)
fmt.Printf("size %d error %v\n", s, e)
})
}
-func iterateEntries(datFile, idxFile *os.File, visitNeedle func(n *needle.Needle, offset int64)) {
+func iterateEntries(datBackend backend.BackendStorageFile, idxFile *os.File, visitNeedle func(n *needle.Needle, offset int64)) {
// start to read index file
var readerOffset int64
bytes := make([]byte, 16)
@@ -78,14 +81,14 @@ func iterateEntries(datFile, idxFile *os.File, visitNeedle func(n *needle.Needle
readerOffset += int64(count)
// start to read dat file
- superBlock, err := storage.ReadSuperBlock(datFile)
+ superBlock, err := super_block.ReadSuperBlock(datBackend)
if err != nil {
fmt.Printf("cannot read dat file super block: %v", err)
return
}
offset := int64(superBlock.BlockSize())
- version := superBlock.Version()
- n, _, rest, err := needle.ReadNeedleHeader(datFile, version, offset)
+ version := superBlock.Version
+ n, _, rest, err := needle.ReadNeedleHeader(datBackend, version, offset)
if err != nil {
fmt.Printf("cannot read needle header: %v", err)
return
@@ -115,7 +118,7 @@ func iterateEntries(datFile, idxFile *os.File, visitNeedle func(n *needle.Needle
fmt.Println("Recovered in f", r)
}
}()
- if _, err = n.ReadNeedleBody(datFile, version, offset+int64(types.NeedleHeaderSize), rest); err != nil {
+ if _, err = n.ReadNeedleBody(datBackend, version, offset+int64(types.NeedleHeaderSize), rest); err != nil {
fmt.Printf("cannot read needle body: offset %d body %d %v\n", offset, rest, err)
}
}()
@@ -127,7 +130,7 @@ func iterateEntries(datFile, idxFile *os.File, visitNeedle func(n *needle.Needle
offset += types.NeedleHeaderSize + rest
//fmt.Printf("==> new entry offset %d\n", offset)
- if n, _, rest, err = needle.ReadNeedleHeader(datFile, version, offset); err != nil {
+ if n, _, rest, err = needle.ReadNeedleHeader(datBackend, version, offset); err != nil {
if err == io.EOF {
return
}
diff --git a/unmaintained/load_test/load_test_leveldb/load_test_leveldb.go b/unmaintained/load_test/load_test_leveldb/load_test_leveldb.go
index 518a5081c..43dfb0e21 100644
--- a/unmaintained/load_test/load_test_leveldb/load_test_leveldb.go
+++ b/unmaintained/load_test/load_test_leveldb/load_test_leveldb.go
@@ -16,7 +16,7 @@ import (
)
var (
- dir = flag.String("dir", "./t", "directory to store level db files")
+ dir = flag.String("dir", "./t", "directory to store level db files")
useHash = flag.Bool("isHash", false, "hash the path as the key")
dbCount = flag.Int("dbCount", 1, "the number of leveldb")
)
@@ -36,7 +36,7 @@ func main() {
var dbs []*leveldb.DB
var chans []chan string
- for d := 0 ; d < *dbCount; d++ {
+ for d := 0; d < *dbCount; d++ {
dbFolder := fmt.Sprintf("%s/%02d", *dir, d)
os.MkdirAll(dbFolder, 0755)
db, err := leveldb.OpenFile(dbFolder, opts)
@@ -49,9 +49,9 @@ func main() {
}
var wg sync.WaitGroup
- for d := 0 ; d < *dbCount; d++ {
+ for d := 0; d < *dbCount; d++ {
wg.Add(1)
- go func(d int){
+ go func(d int) {
defer wg.Done()
ch := chans[d]
@@ -60,14 +60,13 @@ func main() {
for p := range ch {
if *useHash {
insertAsHash(db, p)
- }else{
+ } else {
insertAsFullPath(db, p)
}
}
}(d)
}
-
counter := int64(0)
lastResetTime := time.Now()
@@ -101,7 +100,7 @@ func main() {
}
}
- for d := 0 ; d < *dbCount; d++ {
+ for d := 0; d < *dbCount; d++ {
close(chans[d])
}
diff --git a/unmaintained/remove_duplicate_fids/remove_duplicate_fids.go b/unmaintained/remove_duplicate_fids/remove_duplicate_fids.go
new file mode 100644
index 000000000..84173a663
--- /dev/null
+++ b/unmaintained/remove_duplicate_fids/remove_duplicate_fids.go
@@ -0,0 +1,95 @@
+package main
+
+import (
+ "flag"
+ "fmt"
+ "os"
+ "path/filepath"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+)
+
+var (
+ volumePath = flag.String("dir", "/tmp", "data directory to store files")
+ volumeCollection = flag.String("collection", "", "the volume collection name")
+ volumeId = flag.Int("volumeId", -1, "a volume id. The volume should already exist in the dir. The volume index file should not exist.")
+)
+
+func Checksum(n *needle.Needle) string {
+ return fmt.Sprintf("%s%x", n.Id, n.Cookie)
+}
+
+type VolumeFileScanner4SeeDat struct {
+ version needle.Version
+ block super_block.SuperBlock
+
+ dir string
+ hashes map[string]bool
+ dat *os.File
+ datBackend backend.BackendStorageFile
+}
+
+func (scanner *VolumeFileScanner4SeeDat) VisitSuperBlock(superBlock super_block.SuperBlock) error {
+ scanner.version = superBlock.Version
+ scanner.block = superBlock
+ return nil
+
+}
+func (scanner *VolumeFileScanner4SeeDat) ReadNeedleBody() bool {
+ return true
+}
+
+func (scanner *VolumeFileScanner4SeeDat) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
+
+ if scanner.datBackend == nil {
+ newFileName := filepath.Join(*volumePath, "dat_fixed")
+ newDatFile, err := os.Create(newFileName)
+ if err != nil {
+ glog.Fatalf("Write New Volume Data %v", err)
+ }
+ scanner.datBackend = backend.NewDiskFile(newDatFile)
+ scanner.datBackend.WriteAt(scanner.block.Bytes(), 0)
+ }
+
+ checksum := Checksum(n)
+
+ if scanner.hashes[checksum] {
+ glog.V(0).Infof("duplicate checksum:%s fid:%d,%s%x @ offset:%d", checksum, *volumeId, n.Id, n.Cookie, offset)
+ return nil
+ }
+ scanner.hashes[checksum] = true
+
+ _, s, _, e := n.Append(scanner.datBackend, scanner.version)
+ fmt.Printf("size %d error %v\n", s, e)
+
+ return nil
+}
+
+func main() {
+ flag.Parse()
+
+ vid := needle.VolumeId(*volumeId)
+
+ outpath, _ := filepath.Abs(filepath.Dir(os.Args[0]))
+
+ scanner := &VolumeFileScanner4SeeDat{
+ dir: filepath.Join(outpath, "out"),
+ hashes: map[string]bool{},
+ }
+
+ if _, err := os.Stat(scanner.dir); err != nil {
+ if err := os.MkdirAll(scanner.dir, os.ModePerm); err != nil {
+ glog.Fatalf("could not create output dir : %s", err)
+ }
+ }
+
+ err := storage.ScanVolumeFile(*volumePath, *volumeCollection, vid, storage.NeedleMapInMemory, scanner)
+ if err != nil {
+ glog.Fatalf("Reading Volume File [ERROR] %s\n", err)
+ }
+
+}
diff --git a/unmaintained/see_dat/see_dat.go b/unmaintained/see_dat/see_dat.go
index e8e54fd4f..efc58e751 100644
--- a/unmaintained/see_dat/see_dat.go
+++ b/unmaintained/see_dat/see_dat.go
@@ -2,12 +2,12 @@ package main
import (
"flag"
+ "time"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
-
- "time"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
var (
@@ -20,8 +20,8 @@ type VolumeFileScanner4SeeDat struct {
version needle.Version
}
-func (scanner *VolumeFileScanner4SeeDat) VisitSuperBlock(superBlock storage.SuperBlock) error {
- scanner.version = superBlock.Version()
+func (scanner *VolumeFileScanner4SeeDat) VisitSuperBlock(superBlock super_block.SuperBlock) error {
+ scanner.version = superBlock.Version
return nil
}
@@ -29,7 +29,7 @@ func (scanner *VolumeFileScanner4SeeDat) ReadNeedleBody() bool {
return true
}
-func (scanner *VolumeFileScanner4SeeDat) VisitNeedle(n *needle.Needle, offset int64) error {
+func (scanner *VolumeFileScanner4SeeDat) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
t := time.Unix(int64(n.AppendAtNs)/int64(time.Second), int64(n.AppendAtNs)%int64(time.Second))
glog.V(0).Infof("%d,%s%x offset %d size %d cookie %x appendedAt %v", *volumeId, n.Id, n.Cookie, offset, n.Size, n.Cookie, t)
return nil
@@ -45,5 +45,4 @@ func main() {
if err != nil {
glog.Fatalf("Reading Volume File [ERROR] %s\n", err)
}
-
}
diff --git a/weed/command/backup.go b/weed/command/backup.go
index 31e146965..0f6bed225 100644
--- a/weed/command/backup.go
+++ b/weed/command/backup.go
@@ -3,10 +3,12 @@ package command
import (
"fmt"
+ "github.com/spf13/viper"
+
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/spf13/viper"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/storage"
@@ -17,10 +19,12 @@ var (
)
type BackupOptions struct {
- master *string
- collection *string
- dir *string
- volumeId *int
+ master *string
+ collection *string
+ dir *string
+ volumeId *int
+ ttl *string
+ replication *string
}
func init() {
@@ -29,6 +33,15 @@ func init() {
s.collection = cmdBackup.Flag.String("collection", "", "collection name")
s.dir = cmdBackup.Flag.String("dir", ".", "directory to store volume data files")
s.volumeId = cmdBackup.Flag.Int("volumeId", -1, "a volume id. The volume .dat and .idx files should already exist in the dir.")
+ s.ttl = cmdBackup.Flag.String("ttl", "", `backup volume's time to live, format:
+ 3m: 3 minutes
+ 4h: 4 hours
+ 5d: 5 days
+ 6w: 6 weeks
+ 7M: 7 months
+ 8y: 8 years
+ default is the same with origin`)
+ s.replication = cmdBackup.Flag.String("replication", "", "backup volume's replication, default is the same with origin")
}
var cmdBackup = &Command{
@@ -73,25 +86,42 @@ func runBackup(cmd *Command, args []string) bool {
fmt.Printf("Error get volume %d status: %v\n", vid, err)
return true
}
- ttl, err := needle.ReadTTL(stats.Ttl)
- if err != nil {
- fmt.Printf("Error get volume %d ttl %s: %v\n", vid, stats.Ttl, err)
- return true
+ var ttl *needle.TTL
+ if *s.ttl != "" {
+ ttl, err = needle.ReadTTL(*s.ttl)
+ if err != nil {
+ fmt.Printf("Error generate volume %d ttl %s: %v\n", vid, *s.ttl, err)
+ return true
+ }
+ } else {
+ ttl, err = needle.ReadTTL(stats.Ttl)
+ if err != nil {
+ fmt.Printf("Error get volume %d ttl %s: %v\n", vid, stats.Ttl, err)
+ return true
+ }
}
- replication, err := storage.NewReplicaPlacementFromString(stats.Replication)
- if err != nil {
- fmt.Printf("Error get volume %d replication %s : %v\n", vid, stats.Replication, err)
- return true
+ var replication *super_block.ReplicaPlacement
+ if *s.replication != "" {
+ replication, err = super_block.NewReplicaPlacementFromString(*s.replication)
+ if err != nil {
+ fmt.Printf("Error generate volume %d replication %s : %v\n", vid, *s.replication, err)
+ return true
+ }
+ } else {
+ replication, err = super_block.NewReplicaPlacementFromString(stats.Replication)
+ if err != nil {
+ fmt.Printf("Error get volume %d replication %s : %v\n", vid, stats.Replication, err)
+ return true
+ }
}
-
- v, err := storage.NewVolume(*s.dir, *s.collection, vid, storage.NeedleMapInMemory, replication, ttl, 0)
+ v, err := storage.NewVolume(*s.dir, *s.collection, vid, storage.NeedleMapInMemory, replication, ttl, 0, 0)
if err != nil {
fmt.Printf("Error creating or reading from volume %d: %v\n", vid, err)
return true
}
if v.SuperBlock.CompactionRevision < uint16(stats.CompactRevision) {
- if err = v.Compact(0, 0); err != nil {
+ if err = v.Compact2(30 * 1024 * 1024 * 1024); err != nil {
fmt.Printf("Compact Volume before synchronizing %v\n", err)
return true
}
@@ -100,7 +130,7 @@ func runBackup(cmd *Command, args []string) bool {
return true
}
v.SuperBlock.CompactionRevision = uint16(stats.CompactRevision)
- v.DataFile().WriteAt(v.SuperBlock.Bytes(), 0)
+ v.DataBackend.WriteAt(v.SuperBlock.Bytes(), 0)
}
datSize, _, _ := v.FileStat()
@@ -109,7 +139,7 @@ func runBackup(cmd *Command, args []string) bool {
// remove the old data
v.Destroy()
// recreate an empty volume
- v, err = storage.NewVolume(*s.dir, *s.collection, vid, storage.NeedleMapInMemory, replication, ttl, 0)
+ v, err = storage.NewVolume(*s.dir, *s.collection, vid, storage.NeedleMapInMemory, replication, ttl, 0, 0)
if err != nil {
fmt.Printf("Error creating or reading from volume %d: %v\n", vid, err)
return true
diff --git a/weed/command/benchmark.go b/weed/command/benchmark.go
index dd0fdb88e..26be1fe3a 100644
--- a/weed/command/benchmark.go
+++ b/weed/command/benchmark.go
@@ -69,7 +69,7 @@ func init() {
}
var cmdBenchmark = &Command{
- UsageLine: "benchmark -server=localhost:9333 -c=10 -n=100000",
+ UsageLine: "benchmark -master=localhost:9333 -c=10 -n=100000",
Short: "benchmark on writing millions of files and read out",
Long: `benchmark on an empty SeaweedFS file system.
diff --git a/weed/command/compact.go b/weed/command/compact.go
index 79d50c095..85313b749 100644
--- a/weed/command/compact.go
+++ b/weed/command/compact.go
@@ -17,6 +17,9 @@ var cmdCompact = &Command{
The compacted .dat file is stored as .cpd file.
The compacted .idx file is stored as .cpx file.
+ For method=0, it compacts based on the .dat file, works if .idx file is corrupted.
+ For method=1, it compacts based on the .idx file, works if deletion happened but not written to .dat files.
+
`,
}
@@ -38,7 +41,7 @@ func runCompact(cmd *Command, args []string) bool {
vid := needle.VolumeId(*compactVolumeId)
v, err := storage.NewVolume(*compactVolumePath, *compactVolumeCollection, vid,
- storage.NeedleMapInMemory, nil, nil, preallocate)
+ storage.NeedleMapInMemory, nil, nil, preallocate, 0)
if err != nil {
glog.Fatalf("Load Volume [ERROR] %s\n", err)
}
@@ -47,7 +50,7 @@ func runCompact(cmd *Command, args []string) bool {
glog.Fatalf("Compact Volume [ERROR] %s\n", err)
}
} else {
- if err = v.Compact2(); err != nil {
+ if err = v.Compact2(preallocate); err != nil {
glog.Fatalf("Compact Volume [ERROR] %s\n", err)
}
}
diff --git a/weed/command/export.go b/weed/command/export.go
index 7e94ec11c..8d664ad3b 100644
--- a/weed/command/export.go
+++ b/weed/command/export.go
@@ -4,6 +4,7 @@ import (
"archive/tar"
"bytes"
"fmt"
+ "io"
"os"
"path"
"path/filepath"
@@ -12,11 +13,11 @@ import (
"text/template"
"time"
- "io"
-
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -89,12 +90,12 @@ func printNeedle(vid needle.VolumeId, n *needle.Needle, version needle.Version,
type VolumeFileScanner4Export struct {
version needle.Version
counter int
- needleMap *storage.NeedleMap
+ needleMap *needle_map.MemDb
vid needle.VolumeId
}
-func (scanner *VolumeFileScanner4Export) VisitSuperBlock(superBlock storage.SuperBlock) error {
- scanner.version = superBlock.Version()
+func (scanner *VolumeFileScanner4Export) VisitSuperBlock(superBlock super_block.SuperBlock) error {
+ scanner.version = superBlock.Version
return nil
}
@@ -102,7 +103,7 @@ func (scanner *VolumeFileScanner4Export) ReadNeedleBody() bool {
return true
}
-func (scanner *VolumeFileScanner4Export) VisitNeedle(n *needle.Needle, offset int64) error {
+func (scanner *VolumeFileScanner4Export) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
needleMap := scanner.needleMap
vid := scanner.vid
@@ -192,15 +193,10 @@ func runExport(cmd *Command, args []string) bool {
fileName = *export.collection + "_" + fileName
}
vid := needle.VolumeId(*export.volumeId)
- indexFile, err := os.OpenFile(path.Join(*export.dir, fileName+".idx"), os.O_RDONLY, 0644)
- if err != nil {
- glog.Fatalf("Create Volume Index [ERROR] %s\n", err)
- }
- defer indexFile.Close()
- needleMap, err := storage.LoadBtreeNeedleMap(indexFile)
- if err != nil {
- glog.Fatalf("cannot load needle map from %s: %s", indexFile.Name(), err)
+ needleMap := needle_map.NewMemDb()
+ if err := needleMap.LoadFromIdx(path.Join(*export.dir, fileName+".idx")); err != nil {
+ glog.Fatalf("cannot load needle map from %s.idx: %s", fileName, err)
}
volumeFileScanner := &VolumeFileScanner4Export{
diff --git a/weed/command/filer_copy.go b/weed/command/filer_copy.go
index 19aceb211..f14d18c52 100644
--- a/weed/command/filer_copy.go
+++ b/weed/command/filer_copy.go
@@ -29,16 +29,17 @@ var (
)
type CopyOptions struct {
- include *string
- replication *string
- collection *string
- ttl *string
- maxMB *int
- masterClient *wdclient.MasterClient
- concurrency *int
- compressionLevel *int
- grpcDialOption grpc.DialOption
- masters []string
+ include *string
+ replication *string
+ collection *string
+ ttl *string
+ maxMB *int
+ masterClient *wdclient.MasterClient
+ concurrenctFiles *int
+ concurrenctChunks *int
+ compressionLevel *int
+ grpcDialOption grpc.DialOption
+ masters []string
}
func init() {
@@ -49,7 +50,8 @@ func init() {
copy.collection = cmdCopy.Flag.String("collection", "", "optional collection name")
copy.ttl = cmdCopy.Flag.String("ttl", "", "time to live, e.g.: 1m, 1h, 1d, 1M, 1y")
copy.maxMB = cmdCopy.Flag.Int("maxMB", 32, "split files larger than the limit")
- copy.concurrency = cmdCopy.Flag.Int("c", 8, "concurrent file copy goroutines")
+ copy.concurrenctFiles = cmdCopy.Flag.Int("c", 8, "concurrent file copy goroutines")
+ copy.concurrenctChunks = cmdCopy.Flag.Int("concurrentChunks", 8, "concurrent chunk copy goroutines for each file")
copy.compressionLevel = cmdCopy.Flag.Int("compressionLevel", 9, "local file compression level 1 ~ 9")
}
@@ -131,7 +133,7 @@ func runCopy(cmd *Command, args []string) bool {
util.SetupProfiling("filer.copy.cpu.pprof", "filer.copy.mem.pprof")
}
- fileCopyTaskChan := make(chan FileCopyTask, *copy.concurrency)
+ fileCopyTaskChan := make(chan FileCopyTask, *copy.concurrenctFiles)
go func() {
defer close(fileCopyTaskChan)
@@ -142,7 +144,7 @@ func runCopy(cmd *Command, args []string) bool {
}
}
}()
- for i := 0; i < *copy.concurrency; i++ {
+ for i := 0; i < *copy.concurrenctFiles; i++ {
waitGroup.Add(1)
go func() {
defer waitGroup.Done()
@@ -345,41 +347,71 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
fileName := filepath.Base(f.Name())
mimeType := detectMimeType(f)
- var chunks []*filer_pb.FileChunk
+ chunksChan := make(chan *filer_pb.FileChunk, chunkCount)
+
+ concurrentChunks := make(chan struct{}, *worker.options.concurrenctChunks)
+ var wg sync.WaitGroup
+ var uploadError error
+
+ fmt.Printf("uploading %s in %d chunks ...\n", fileName, chunkCount)
+ for i := int64(0); i < int64(chunkCount) && uploadError == nil; i++ {
+ wg.Add(1)
+ concurrentChunks <- struct{}{}
+ go func(i int64) {
+ defer func() {
+ wg.Done()
+ <-concurrentChunks
+ }()
+ // assign a volume
+ assignResult, err := operation.Assign(worker.options.masterClient.GetMaster(), worker.options.grpcDialOption, &operation.VolumeAssignRequest{
+ Count: 1,
+ Replication: *worker.options.replication,
+ Collection: *worker.options.collection,
+ Ttl: *worker.options.ttl,
+ })
+ if err != nil {
+ fmt.Printf("Failed to assign from %v: %v\n", worker.options.masters, err)
+ }
- for i := int64(0); i < int64(chunkCount); i++ {
+ targetUrl := "http://" + assignResult.Url + "/" + assignResult.Fid
- // assign a volume
- assignResult, err := operation.Assign(worker.options.masterClient.GetMaster(), worker.options.grpcDialOption, &operation.VolumeAssignRequest{
- Count: 1,
- Replication: *worker.options.replication,
- Collection: *worker.options.collection,
- Ttl: *worker.options.ttl,
- })
- if err != nil {
- fmt.Printf("Failed to assign from %v: %v\n", worker.options.masters, err)
- }
+ uploadResult, err := operation.Upload(targetUrl,
+ fileName+"-"+strconv.FormatInt(i+1, 10),
+ io.NewSectionReader(f, i*chunkSize, chunkSize),
+ false, "", nil, assignResult.Auth)
+ if err != nil {
+ uploadError = fmt.Errorf("upload data %v to %s: %v\n", fileName, targetUrl, err)
+ return
+ }
+ if uploadResult.Error != "" {
+ uploadError = fmt.Errorf("upload %v to %s result: %v\n", fileName, targetUrl, uploadResult.Error)
+ return
+ }
+ chunksChan <- &filer_pb.FileChunk{
+ FileId: assignResult.Fid,
+ Offset: i * chunkSize,
+ Size: uint64(uploadResult.Size),
+ Mtime: time.Now().UnixNano(),
+ ETag: uploadResult.ETag,
+ }
+ fmt.Printf("uploaded %s-%d to %s [%d,%d)\n", fileName, i+1, targetUrl, i*chunkSize, i*chunkSize+int64(uploadResult.Size))
+ }(i)
+ }
+ wg.Wait()
+ close(chunksChan)
- targetUrl := "http://" + assignResult.Url + "/" + assignResult.Fid
+ var chunks []*filer_pb.FileChunk
+ for chunk := range chunksChan {
+ chunks = append(chunks, chunk)
+ }
- uploadResult, err := operation.Upload(targetUrl,
- fileName+"-"+strconv.FormatInt(i+1, 10),
- io.LimitReader(f, chunkSize),
- false, "application/octet-stream", nil, assignResult.Auth)
- if err != nil {
- return fmt.Errorf("upload data %v to %s: %v\n", fileName, targetUrl, err)
- }
- if uploadResult.Error != "" {
- return fmt.Errorf("upload %v to %s result: %v\n", fileName, targetUrl, uploadResult.Error)
+ if uploadError != nil {
+ var fileIds []string
+ for _, chunk := range chunks {
+ fileIds = append(fileIds, chunk.FileId)
}
- chunks = append(chunks, &filer_pb.FileChunk{
- FileId: assignResult.Fid,
- Offset: i * chunkSize,
- Size: uint64(uploadResult.Size),
- Mtime: time.Now().UnixNano(),
- ETag: uploadResult.ETag,
- })
- fmt.Printf("uploaded %s-%d to %s [%d,%d)\n", fileName, i+1, targetUrl, i*chunkSize, i*chunkSize+int64(uploadResult.Size))
+ operation.DeleteFiles(worker.options.masterClient.GetMaster(), worker.options.grpcDialOption, fileIds)
+ return uploadError
}
if err := withFilerClient(ctx, worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
diff --git a/weed/command/fix.go b/weed/command/fix.go
index bf33490cc..76bc19f7e 100644
--- a/weed/command/fix.go
+++ b/weed/command/fix.go
@@ -8,6 +8,8 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -31,11 +33,11 @@ var (
type VolumeFileScanner4Fix struct {
version needle.Version
- nm *storage.NeedleMap
+ nm *needle_map.MemDb
}
-func (scanner *VolumeFileScanner4Fix) VisitSuperBlock(superBlock storage.SuperBlock) error {
- scanner.version = superBlock.Version()
+func (scanner *VolumeFileScanner4Fix) VisitSuperBlock(superBlock super_block.SuperBlock) error {
+ scanner.version = superBlock.Version
return nil
}
@@ -43,14 +45,14 @@ func (scanner *VolumeFileScanner4Fix) ReadNeedleBody() bool {
return false
}
-func (scanner *VolumeFileScanner4Fix) VisitNeedle(n *needle.Needle, offset int64) error {
+func (scanner *VolumeFileScanner4Fix) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
glog.V(2).Infof("key %d offset %d size %d disk_size %d gzip %v", n.Id, offset, n.Size, n.DiskSize(scanner.version), n.IsGzipped())
if n.Size > 0 && n.Size != types.TombstoneFileSize {
- pe := scanner.nm.Put(n.Id, types.ToOffset(offset), n.Size)
+ pe := scanner.nm.Set(n.Id, types.ToOffset(offset), n.Size)
glog.V(2).Infof("saved %d with error %v", n.Size, pe)
} else {
glog.V(2).Infof("skipping deleted file ...")
- return scanner.nm.Delete(n.Id, types.ToOffset(offset))
+ return scanner.nm.Delete(n.Id)
}
return nil
}
@@ -72,8 +74,7 @@ func runFix(cmd *Command, args []string) bool {
}
defer indexFile.Close()
- nm := storage.NewBtreeNeedleMap(indexFile)
- defer nm.Close()
+ nm := needle_map.NewMemDb()
vid := needle.VolumeId(*fixVolumeId)
scanner := &VolumeFileScanner4Fix{
diff --git a/weed/command/master.go b/weed/command/master.go
index 9e9308468..8d0a3289c 100644
--- a/weed/command/master.go
+++ b/weed/command/master.go
@@ -12,6 +12,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/server"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/util"
"github.com/gorilla/mux"
"github.com/spf13/viper"
@@ -72,8 +73,6 @@ var cmdMaster = &Command{
var (
masterCpuProfile = cmdMaster.Flag.String("cpuprofile", "", "cpu profile output file")
masterMemProfile = cmdMaster.Flag.String("memprofile", "", "memory profile output file")
-
- masterWhiteList []string
)
func runMaster(cmd *Command, args []string) bool {
@@ -87,6 +86,8 @@ func runMaster(cmd *Command, args []string) bool {
if err := util.TestFolderWritable(*m.metaFolder); err != nil {
glog.Fatalf("Check Meta Folder (-mdir) Writable %s : %s", *m.metaFolder, err)
}
+
+ var masterWhiteList []string
if *m.whiteList != "" {
masterWhiteList = strings.Split(*m.whiteList, ",")
}
@@ -94,52 +95,54 @@ func runMaster(cmd *Command, args []string) bool {
glog.Fatalf("volumeSizeLimitMB should be smaller than 30000")
}
- r := mux.NewRouter()
- ms := weed_server.NewMasterServer(r, m.toMasterOption(masterWhiteList))
+ startMaster(m, masterWhiteList)
+
+ return true
+}
- listeningAddress := *m.ipBind + ":" + strconv.Itoa(*m.port)
+func startMaster(masterOption MasterOptions, masterWhiteList []string) {
- glog.V(0).Infoln("Start Seaweed Master", util.VERSION, "at", listeningAddress)
+ backend.LoadConfiguration(viper.GetViper())
+ myMasterAddress, peers := checkPeers(*masterOption.ip, *masterOption.port, *masterOption.peers)
+
+ r := mux.NewRouter()
+ ms := weed_server.NewMasterServer(r, masterOption.toMasterOption(masterWhiteList), peers)
+ listeningAddress := *masterOption.ipBind + ":" + strconv.Itoa(*masterOption.port)
+ glog.V(0).Infof("Start Seaweed Master %s at %s", util.VERSION, listeningAddress)
masterListener, e := util.NewListener(listeningAddress, 0)
if e != nil {
glog.Fatalf("Master startup error: %v", e)
}
+ // start raftServer
+ raftServer := weed_server.NewRaftServer(security.LoadClientTLS(viper.Sub("grpc"), "master"),
+ peers, myMasterAddress, *masterOption.metaFolder, ms.Topo, *masterOption.pulseSeconds)
+ if raftServer == nil {
+ glog.Fatalf("please verify %s is writable, see https://github.com/chrislusf/seaweedfs/issues/717", *masterOption.metaFolder)
+ }
+ ms.SetRaftServer(raftServer)
+ r.HandleFunc("/cluster/status", raftServer.StatusHandler).Methods("GET")
+ // starting grpc server
+ grpcPort := *masterOption.port + 10000
+ grpcL, err := util.NewListener(*masterOption.ipBind+":"+strconv.Itoa(grpcPort), 0)
+ if err != nil {
+ glog.Fatalf("master failed to listen on grpc port %d: %v", grpcPort, err)
+ }
+ // Create your protocol servers.
+ grpcS := util.NewGrpcServer(security.LoadServerTLS(viper.Sub("grpc"), "master"))
+ master_pb.RegisterSeaweedServer(grpcS, ms)
+ protobuf.RegisterRaftServer(grpcS, raftServer)
+ reflection.Register(grpcS)
+ glog.V(0).Infof("Start Seaweed Master %s grpc server at %s:%d", util.VERSION, *masterOption.ipBind, grpcPort)
+ go grpcS.Serve(grpcL)
- go func() {
- // start raftServer
- myMasterAddress, peers := checkPeers(*m.ip, *m.port, *m.peers)
- raftServer := weed_server.NewRaftServer(security.LoadClientTLS(viper.Sub("grpc"), "master"),
- peers, myMasterAddress, *m.metaFolder, ms.Topo, *m.pulseSeconds)
- if raftServer == nil {
- glog.Fatalf("please verify %s is writable, see https://github.com/chrislusf/seaweedfs/issues/717", *m.metaFolder)
- }
- ms.SetRaftServer(raftServer)
- r.HandleFunc("/cluster/status", raftServer.StatusHandler).Methods("GET")
-
- // starting grpc server
- grpcPort := *m.port + 10000
- grpcL, err := util.NewListener(*m.ipBind+":"+strconv.Itoa(grpcPort), 0)
- if err != nil {
- glog.Fatalf("master failed to listen on grpc port %d: %v", grpcPort, err)
- }
- // Create your protocol servers.
- grpcS := util.NewGrpcServer(security.LoadServerTLS(viper.Sub("grpc"), "master"))
- master_pb.RegisterSeaweedServer(grpcS, ms)
- protobuf.RegisterRaftServer(grpcS, raftServer)
- reflection.Register(grpcS)
-
- glog.V(0).Infof("Start Seaweed Master %s grpc server at %s:%d", util.VERSION, *m.ipBind, grpcPort)
- grpcS.Serve(grpcL)
- }()
+ go ms.MasterClient.KeepConnectedToMaster()
// start http server
httpS := &http.Server{Handler: r}
- if err := httpS.Serve(masterListener); err != nil {
- glog.Fatalf("master server failed to serve: %v", err)
- }
+ go httpS.Serve(masterListener)
- return true
+ select {}
}
func checkPeers(masterIp string, masterPort int, peers string) (masterAddress string, cleanedPeers []string) {
@@ -156,11 +159,10 @@ func checkPeers(masterIp string, masterPort int, peers string) (masterAddress st
}
}
- peerCount := len(cleanedPeers)
if !hasSelf {
- peerCount += 1
+ cleanedPeers = append(cleanedPeers, masterAddress)
}
- if peerCount%2 == 0 {
+ if len(cleanedPeers)%2 == 0 {
glog.Fatalf("Only odd number of masters are supported!")
}
return
diff --git a/weed/command/mount.go b/weed/command/mount.go
index ec790c999..f09b285f7 100644
--- a/weed/command/mount.go
+++ b/weed/command/mount.go
@@ -10,13 +10,14 @@ type MountOptions struct {
filer *string
filerMountRootPath *string
dir *string
- dirListingLimit *int
+ dirListCacheLimit *int64
collection *string
replication *string
ttlSec *int
chunkSizeLimitMB *int
dataCenter *string
allowOthers *bool
+ umaskString *string
}
var (
@@ -30,13 +31,14 @@ func init() {
mountOptions.filer = cmdMount.Flag.String("filer", "localhost:8888", "weed filer location")
mountOptions.filerMountRootPath = cmdMount.Flag.String("filer.path", "/", "mount this remote path from filer server")
mountOptions.dir = cmdMount.Flag.String("dir", ".", "mount weed filer to this directory")
- mountOptions.dirListingLimit = cmdMount.Flag.Int("dirListLimit", 100000, "limit directory listing size")
+ mountOptions.dirListCacheLimit = cmdMount.Flag.Int64("dirListCacheLimit", 1000000, "limit cache size to speed up directory long format listing")
mountOptions.collection = cmdMount.Flag.String("collection", "", "collection to create the files")
mountOptions.replication = cmdMount.Flag.String("replication", "", "replication(e.g. 000, 001) to create to files. If empty, let filer decide.")
mountOptions.ttlSec = cmdMount.Flag.Int("ttl", 0, "file ttl in seconds")
mountOptions.chunkSizeLimitMB = cmdMount.Flag.Int("chunkSizeLimitMB", 4, "local write buffer size, also chunk large files")
mountOptions.dataCenter = cmdMount.Flag.String("dataCenter", "", "prefer to write to the data center")
mountOptions.allowOthers = cmdMount.Flag.Bool("allowOthers", true, "allows other users to access the file system")
+ mountOptions.umaskString = cmdMount.Flag.String("umask", "022", "octal umask, e.g., 022, 0111")
mountCpuProfile = cmdMount.Flag.String("cpuprofile", "", "cpu profile output file")
mountMemProfile = cmdMount.Flag.String("memprofile", "", "memory profile output file")
}
@@ -62,12 +64,12 @@ var cmdMount = &Command{
func parseFilerGrpcAddress(filer string) (filerGrpcAddress string, err error) {
hostnameAndPort := strings.Split(filer, ":")
if len(hostnameAndPort) != 2 {
- return "", fmt.Errorf("The filer should have hostname:port format: %v", hostnameAndPort)
+ return "", fmt.Errorf("filer should have hostname:port format: %v", hostnameAndPort)
}
filerPort, parseErr := strconv.ParseUint(hostnameAndPort[1], 10, 64)
if parseErr != nil {
- return "", fmt.Errorf("The filer filer port parse error: %v", parseErr)
+ return "", fmt.Errorf("filer port parse error: %v", parseErr)
}
filerGrpcPort := int(filerPort) + 10000
diff --git a/weed/command/mount_darwin.go b/weed/command/mount_darwin.go
new file mode 100644
index 000000000..f0a5581e7
--- /dev/null
+++ b/weed/command/mount_darwin.go
@@ -0,0 +1,13 @@
+package command
+
+import (
+ "github.com/seaweedfs/fuse"
+)
+
+func osSpecificMountOptions() []fuse.MountOption {
+ return []fuse.MountOption{}
+}
+
+func checkMountPointAvailable(dir string) bool {
+ return true
+}
diff --git a/weed/command/mount_freebsd.go b/weed/command/mount_freebsd.go
new file mode 100644
index 000000000..f0a5581e7
--- /dev/null
+++ b/weed/command/mount_freebsd.go
@@ -0,0 +1,13 @@
+package command
+
+import (
+ "github.com/seaweedfs/fuse"
+)
+
+func osSpecificMountOptions() []fuse.MountOption {
+ return []fuse.MountOption{}
+}
+
+func checkMountPointAvailable(dir string) bool {
+ return true
+}
diff --git a/weed/command/mount_linux.go b/weed/command/mount_linux.go
new file mode 100644
index 000000000..80a5f9da4
--- /dev/null
+++ b/weed/command/mount_linux.go
@@ -0,0 +1,157 @@
+package command
+
+import (
+ "bufio"
+ "fmt"
+ "io"
+ "os"
+ "strings"
+
+ "github.com/seaweedfs/fuse"
+)
+
+const (
+ /* 36 35 98:0 /mnt1 /mnt2 rw,noatime master:1 - ext3 /dev/root rw,errors=continue
+ (1)(2)(3) (4) (5) (6) (7) (8) (9) (10) (11)
+
+ (1) mount ID: unique identifier of the mount (may be reused after umount)
+ (2) parent ID: ID of parent (or of self for the top of the mount tree)
+ (3) major:minor: value of st_dev for files on filesystem
+ (4) root: root of the mount within the filesystem
+ (5) mount point: mount point relative to the process's root
+ (6) mount options: per mount options
+ (7) optional fields: zero or more fields of the form "tag[:value]"
+ (8) separator: marks the end of the optional fields
+ (9) filesystem type: name of filesystem of the form "type[.subtype]"
+ (10) mount source: filesystem specific information or "none"
+ (11) super options: per super block options*/
+ mountinfoFormat = "%d %d %d:%d %s %s %s %s"
+)
+
+// Info reveals information about a particular mounted filesystem. This
+// struct is populated from the content in the /proc//mountinfo file.
+type Info struct {
+ // ID is a unique identifier of the mount (may be reused after umount).
+ ID int
+
+ // Parent indicates the ID of the mount parent (or of self for the top of the
+ // mount tree).
+ Parent int
+
+ // Major indicates one half of the device ID which identifies the device class.
+ Major int
+
+ // Minor indicates one half of the device ID which identifies a specific
+ // instance of device.
+ Minor int
+
+ // Root of the mount within the filesystem.
+ Root string
+
+ // Mountpoint indicates the mount point relative to the process's root.
+ Mountpoint string
+
+ // Opts represents mount-specific options.
+ Opts string
+
+ // Optional represents optional fields.
+ Optional string
+
+ // Fstype indicates the type of filesystem, such as EXT3.
+ Fstype string
+
+ // Source indicates filesystem specific information or "none".
+ Source string
+
+ // VfsOpts represents per super block options.
+ VfsOpts string
+}
+
+// Mounted determines if a specified mountpoint has been mounted.
+// On Linux it looks at /proc/self/mountinfo and on Solaris at mnttab.
+func mounted(mountPoint string) (bool, error) {
+ entries, err := parseMountTable()
+ if err != nil {
+ return false, err
+ }
+
+ // Search the table for the mountPoint
+ for _, e := range entries {
+ if e.Mountpoint == mountPoint {
+ return true, nil
+ }
+ }
+ return false, nil
+}
+
+// Parse /proc/self/mountinfo because comparing Dev and ino does not work from
+// bind mounts
+func parseMountTable() ([]*Info, error) {
+ f, err := os.Open("/proc/self/mountinfo")
+ if err != nil {
+ return nil, err
+ }
+ defer f.Close()
+
+ return parseInfoFile(f)
+}
+
+func parseInfoFile(r io.Reader) ([]*Info, error) {
+ var (
+ s = bufio.NewScanner(r)
+ out []*Info
+ )
+
+ for s.Scan() {
+ if err := s.Err(); err != nil {
+ return nil, err
+ }
+
+ var (
+ p = &Info{}
+ text = s.Text()
+ optionalFields string
+ )
+
+ if _, err := fmt.Sscanf(text, mountinfoFormat,
+ &p.ID, &p.Parent, &p.Major, &p.Minor,
+ &p.Root, &p.Mountpoint, &p.Opts, &optionalFields); err != nil {
+ return nil, fmt.Errorf("Scanning '%s' failed: %s", text, err)
+ }
+ // Safe as mountinfo encodes mountpoints with spaces as \040.
+ index := strings.Index(text, " - ")
+ postSeparatorFields := strings.Fields(text[index+3:])
+ if len(postSeparatorFields) < 3 {
+ return nil, fmt.Errorf("Error found less than 3 fields post '-' in %q", text)
+ }
+
+ if optionalFields != "-" {
+ p.Optional = optionalFields
+ }
+
+ p.Fstype = postSeparatorFields[0]
+ p.Source = postSeparatorFields[1]
+ p.VfsOpts = strings.Join(postSeparatorFields[2:], " ")
+ out = append(out, p)
+ }
+ return out, nil
+}
+
+func osSpecificMountOptions() []fuse.MountOption {
+ return []fuse.MountOption{
+ fuse.AllowNonEmptyMount(),
+ }
+}
+
+func checkMountPointAvailable(dir string) bool {
+ mountPoint := dir
+ if mountPoint != "/" && strings.HasSuffix(mountPoint, "/") {
+ mountPoint = mountPoint[0 : len(mountPoint)-1]
+ }
+
+ if mounted, err := mounted(mountPoint); err != nil || mounted {
+ return false
+ }
+
+ return true
+}
diff --git a/weed/command/mount_notsupported.go b/weed/command/mount_notsupported.go
index 3bf22ddc4..f3c0de3d6 100644
--- a/weed/command/mount_notsupported.go
+++ b/weed/command/mount_notsupported.go
@@ -1,5 +1,6 @@
// +build !linux
// +build !darwin
+// +build !freebsd
package command
diff --git a/weed/command/mount_std.go b/weed/command/mount_std.go
index 1d1214266..453531d00 100644
--- a/weed/command/mount_std.go
+++ b/weed/command/mount_std.go
@@ -1,4 +1,4 @@
-// +build linux darwin
+// +build linux darwin freebsd
package command
@@ -12,12 +12,12 @@ import (
"strings"
"time"
- "github.com/chrislusf/seaweedfs/weed/security"
"github.com/jacobsa/daemonize"
"github.com/spf13/viper"
"github.com/chrislusf/seaweedfs/weed/filesys"
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/util"
"github.com/seaweedfs/fuse"
"github.com/seaweedfs/fuse/fs"
@@ -27,6 +27,12 @@ func runMount(cmd *Command, args []string) bool {
util.SetupProfiling(*mountCpuProfile, *mountMemProfile)
+ umask, umaskErr := strconv.ParseUint(*mountOptions.umaskString, 8, 64)
+ if umaskErr != nil {
+ fmt.Printf("can not parse umask %s", *mountOptions.umaskString)
+ return false
+ }
+
return RunMount(
*mountOptions.filer,
*mountOptions.filerMountRootPath,
@@ -37,12 +43,13 @@ func runMount(cmd *Command, args []string) bool {
*mountOptions.chunkSizeLimitMB,
*mountOptions.allowOthers,
*mountOptions.ttlSec,
- *mountOptions.dirListingLimit,
+ *mountOptions.dirListCacheLimit,
+ os.FileMode(umask),
)
}
func RunMount(filer, filerMountRootPath, dir, collection, replication, dataCenter string, chunkSizeLimitMB int,
- allowOthers bool, ttlSec int, dirListingLimit int) bool {
+ allowOthers bool, ttlSec int, dirListCacheLimit int64, umask os.FileMode) bool {
util.LoadConfiguration("security", false)
@@ -81,12 +88,18 @@ func RunMount(filer, filerMountRootPath, dir, collection, replication, dataCente
}
}
+ // Ensure target mount point availability
+ if isValid := checkMountPointAvailable(dir); !isValid {
+ glog.Fatalf("Expected mount to still be active, target mount point: %s, please check!", dir)
+ return false
+ }
+
mountName := path.Base(dir)
options := []fuse.MountOption{
fuse.VolumeName(mountName),
- fuse.FSName("SeaweedFS"),
- fuse.Subtype("SeaweedFS"),
+ fuse.FSName(filer + ":" + filerMountRootPath),
+ fuse.Subtype("seaweedfs"),
fuse.NoAppleDouble(),
fuse.NoAppleXattr(),
fuse.NoBrowse(),
@@ -100,15 +113,18 @@ func RunMount(filer, filerMountRootPath, dir, collection, replication, dataCente
fuse.WritebackCache(),
fuse.AllowNonEmptyMount(),
}
+
+ options = append(options, osSpecificMountOptions()...)
+
if allowOthers {
options = append(options, fuse.AllowOther())
}
c, err := fuse.Mount(dir, options...)
if err != nil {
- glog.Fatal(err)
+ glog.V(0).Infof("mount: %v", err)
daemonize.SignalOutcome(err)
- return false
+ return true
}
util.OnInterrupt(func() {
@@ -118,9 +134,9 @@ func RunMount(filer, filerMountRootPath, dir, collection, replication, dataCente
filerGrpcAddress, err := parseFilerGrpcAddress(filer)
if err != nil {
- glog.Fatal(err)
+ glog.V(0).Infof("parseFilerGrpcAddress: %v", err)
daemonize.SignalOutcome(err)
- return false
+ return true
}
mountRoot := filerMountRootPath
@@ -139,13 +155,14 @@ func RunMount(filer, filerMountRootPath, dir, collection, replication, dataCente
TtlSec: int32(ttlSec),
ChunkSizeLimit: int64(chunkSizeLimitMB) * 1024 * 1024,
DataCenter: dataCenter,
- DirListingLimit: dirListingLimit,
+ DirListCacheLimit: dirListCacheLimit,
EntryCacheTtl: 3 * time.Second,
MountUid: uid,
MountGid: gid,
MountMode: mountMode,
MountCtime: fileInfo.ModTime(),
MountMtime: time.Now(),
+ Umask: umask,
}))
if err != nil {
fuse.Unmount(dir)
@@ -154,8 +171,9 @@ func RunMount(filer, filerMountRootPath, dir, collection, replication, dataCente
// check if the mount process has an error to report
<-c.Ready
if err := c.MountError; err != nil {
- glog.Fatal(err)
+ glog.V(0).Infof("mount process: %v", err)
daemonize.SignalOutcome(err)
+ return true
}
return true
diff --git a/weed/command/scaffold.go b/weed/command/scaffold.go
index 062fe0ff8..a76466ed6 100644
--- a/weed/command/scaffold.go
+++ b/weed/command/scaffold.go
@@ -59,15 +59,6 @@ const (
# $HOME/.seaweedfs/filer.toml
# /etc/seaweedfs/filer.toml
-[memory]
-# local in memory, mostly for testing purpose
-enabled = false
-
-[leveldb]
-# local on disk, mostly for simple single-machine setup, fairly scalable
-enabled = false
-dir = "." # directory to store level db files
-
[leveldb2]
# local on disk, mostly for simple single-machine setup, fairly scalable
# faster than previous leveldb, recommended.
@@ -78,7 +69,7 @@ dir = "." # directory to store level db files
# multiple filers on shared storage, fairly scalable
####################################################
-[mysql]
+[mysql] # or tidb
# CREATE TABLE IF NOT EXISTS filemeta (
# dirhash BIGINT COMMENT 'first 64 bits of MD5 hash value of directory field',
# name VARCHAR(1000) COMMENT 'directory or file name',
@@ -95,8 +86,9 @@ password = ""
database = "" # create or use an existing database
connection_max_idle = 2
connection_max_open = 100
+interpolateParams = false
-[postgres]
+[postgres] # or cockroachdb
# CREATE TABLE IF NOT EXISTS filemeta (
# dirhash BIGINT,
# name VARCHAR(65535),
@@ -131,7 +123,7 @@ hosts=[
enabled = false
address = "localhost:6379"
password = ""
-db = 0
+database = 0
[redis_cluster]
enabled = false
@@ -144,6 +136,20 @@ addresses = [
"localhost:30006",
]
password = ""
+// allows reads from slave servers or the master, but all writes still go to the master
+readOnly = true
+// automatically use the closest Redis server for reads
+routeByLatency = true
+
+[etcd]
+enabled = false
+servers = "localhost:2379"
+timeout = "3s"
+
+[tikv]
+enabled = false
+pdAddress = "192.168.199.113:2379"
+
`
@@ -217,22 +223,22 @@ grpcAddress = "localhost:18888"
# all files under this directory tree are replicated.
# this is not a directory on your hard drive, but on your filer.
# i.e., all files with this "prefix" are sent to notification message queue.
-directory = "/buckets"
+directory = "/buckets"
[sink.filer]
enabled = false
grpcAddress = "localhost:18888"
# all replicated files are under this directory tree
-# this is not a directory on your hard drive, but on your filer.
+# this is not a directory on your hard drive, but on your filer.
# i.e., all received files will be "prefixed" to this directory.
-directory = "/backup"
+directory = "/backup"
replication = ""
collection = ""
ttlSec = 0
[sink.s3]
# read credentials doc at https://docs.aws.amazon.com/sdk-for-go/v1/developer-guide/sessions.html
-# default loads credentials from the shared credentials file (~/.aws/credentials).
+# default loads credentials from the shared credentials file (~/.aws/credentials).
enabled = false
aws_access_key_id = "" # if empty, loads from the shared credentials file (~/.aws/credentials).
aws_secret_access_key = "" # if empty, loads from the shared credentials file (~/.aws/credentials).
@@ -336,5 +342,23 @@ scripts = """
"""
sleep_minutes = 17 # sleep minutes between each script execution
+[master.filer]
+default_filer_url = "http://localhost:8888/"
+
+[master.sequencer]
+type = "memory" # Choose [memory|etcd] type for storing the file id sequence
+# when sequencer.type = etcd, set listen client urls of etcd cluster that store file id sequence
+# example : http://127.0.0.1:2379,http://127.0.0.1:2389
+sequencer_etcd_urls = "http://127.0.0.1:2379"
+
+
+[storage.backend]
+ [storage.backend.s3.default]
+ enabled = false
+ aws_access_key_id = "" # if empty, loads from the shared credentials file (~/.aws/credentials).
+ aws_secret_access_key = "" # if empty, loads from the shared credentials file (~/.aws/credentials).
+ region = "us-east-2"
+ bucket = "your_bucket_name" # an existing bucket
+
`
)
diff --git a/weed/command/scaffold_test.go b/weed/command/scaffold_test.go
new file mode 100644
index 000000000..423dacc32
--- /dev/null
+++ b/weed/command/scaffold_test.go
@@ -0,0 +1,44 @@
+package command
+
+import (
+ "bytes"
+ "fmt"
+ "testing"
+
+ "github.com/spf13/viper"
+)
+
+func TestReadingTomlConfiguration(t *testing.T) {
+
+ viper.SetConfigType("toml")
+
+ // any approach to require this configuration into your program.
+ var tomlExample = []byte(`
+[database]
+server = "192.168.1.1"
+ports = [ 8001, 8001, 8002 ]
+connection_max = 5000
+enabled = true
+
+[servers]
+
+ # You can indent as you please. Tabs or spaces. TOML don't care.
+ [servers.alpha]
+ ip = "10.0.0.1"
+ dc = "eqdc10"
+
+ [servers.beta]
+ ip = "10.0.0.2"
+ dc = "eqdc10"
+
+`)
+
+ viper.ReadConfig(bytes.NewBuffer(tomlExample))
+
+ fmt.Printf("database is %v\n", viper.Get("database"))
+ fmt.Printf("servers is %v\n", viper.GetStringMap("servers"))
+
+ alpha := viper.Sub("servers.alpha")
+
+ fmt.Printf("alpha ip is %v\n", alpha.GetString("ip"))
+}
diff --git a/weed/command/server.go b/weed/command/server.go
index f8c1d06fc..87f404ed3 100644
--- a/weed/command/server.go
+++ b/weed/command/server.go
@@ -2,25 +2,14 @@ package command
import (
"fmt"
- "net/http"
"os"
"runtime"
"runtime/pprof"
- "strconv"
"strings"
- "sync"
"time"
- "github.com/chrislusf/raft/protobuf"
- "github.com/chrislusf/seaweedfs/weed/security"
- "github.com/spf13/viper"
-
"github.com/chrislusf/seaweedfs/weed/glog"
- "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
- "github.com/chrislusf/seaweedfs/weed/server"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/gorilla/mux"
- "google.golang.org/grpc/reflection"
)
type ServerOptions struct {
@@ -132,14 +121,17 @@ func runServer(cmd *Command, args []string) bool {
*isStartingFiler = true
}
- master := *serverIp + ":" + strconv.Itoa(*masterOptions.port)
+ _, peerList := checkPeers(*serverIp, *masterOptions.port, *masterOptions.peers)
+ peers := strings.Join(peerList, ",")
+ masterOptions.peers = &peers
+
masterOptions.ip = serverIp
masterOptions.ipBind = serverBindIp
- filerOptions.masters = &master
+ filerOptions.masters = &peers
filerOptions.ip = serverBindIp
serverOptions.v.ip = serverIp
serverOptions.v.bindIp = serverBindIp
- serverOptions.v.masters = &master
+ serverOptions.v.masters = &peers
serverOptions.v.idleConnectionTimeout = serverTimeout
serverOptions.v.dataCenter = serverDataCenter
serverOptions.v.rack = serverRack
@@ -198,59 +190,12 @@ func runServer(cmd *Command, args []string) bool {
}()
}
- var volumeWait sync.WaitGroup
-
- volumeWait.Add(1)
-
- go func() {
- r := mux.NewRouter()
- ms := weed_server.NewMasterServer(r, masterOptions.toMasterOption(serverWhiteList))
-
- glog.V(0).Infof("Start Seaweed Master %s at %s:%d", util.VERSION, *serverIp, *masterOptions.port)
- masterListener, e := util.NewListener(*serverBindIp+":"+strconv.Itoa(*masterOptions.port), 0)
- if e != nil {
- glog.Fatalf("Master startup error: %v", e)
- }
-
- go func() {
- // start raftServer
- myMasterAddress, peers := checkPeers(*serverIp, *masterOptions.port, *masterOptions.peers)
- raftServer := weed_server.NewRaftServer(security.LoadClientTLS(viper.Sub("grpc"), "master"),
- peers, myMasterAddress, *masterOptions.metaFolder, ms.Topo, *masterOptions.pulseSeconds)
- ms.SetRaftServer(raftServer)
- r.HandleFunc("/cluster/status", raftServer.StatusHandler).Methods("GET")
-
- // starting grpc server
- grpcPort := *masterOptions.port + 10000
- grpcL, err := util.NewListener(*serverBindIp+":"+strconv.Itoa(grpcPort), 0)
- if err != nil {
- glog.Fatalf("master failed to listen on grpc port %d: %v", grpcPort, err)
- }
- // Create your protocol servers.
- glog.V(1).Infof("grpc config %+v", viper.Sub("grpc"))
- grpcS := util.NewGrpcServer(security.LoadServerTLS(viper.Sub("grpc"), "master"))
- master_pb.RegisterSeaweedServer(grpcS, ms)
- protobuf.RegisterRaftServer(grpcS, raftServer)
- reflection.Register(grpcS)
-
- glog.V(0).Infof("Start Seaweed Master %s grpc server at %s:%d", util.VERSION, *serverIp, grpcPort)
- grpcS.Serve(grpcL)
- }()
-
- volumeWait.Done()
-
- // start http server
- httpS := &http.Server{Handler: r}
- if err := httpS.Serve(masterListener); err != nil {
- glog.Fatalf("master server failed to serve: %v", err)
- }
-
- }()
-
- volumeWait.Wait()
- time.Sleep(100 * time.Millisecond)
+ // start volume server
+ {
+ go serverOptions.v.startVolumeServer(*volumeDataFolders, *volumeMaxDataVolumeCounts, *serverWhiteListOption)
+ }
- serverOptions.v.startVolumeServer(*volumeDataFolders, *volumeMaxDataVolumeCounts, *serverWhiteListOption)
+ startMaster(masterOptions, serverWhiteList)
return true
}
diff --git a/weed/command/shell.go b/weed/command/shell.go
index 79f8b8bf9..34b5aef31 100644
--- a/weed/command/shell.go
+++ b/weed/command/shell.go
@@ -1,6 +1,8 @@
package command
import (
+ "fmt"
+
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/shell"
"github.com/chrislusf/seaweedfs/weed/util"
@@ -8,12 +10,14 @@ import (
)
var (
- shellOptions shell.ShellOptions
+ shellOptions shell.ShellOptions
+ shellInitialFilerUrl *string
)
func init() {
cmdShell.Run = runShell // break init cycle
shellOptions.Masters = cmdShell.Flag.String("master", "localhost:9333", "comma-separated master servers")
+ shellInitialFilerUrl = cmdShell.Flag.String("filer.url", "http://localhost:8888/", "initial filer url")
}
var cmdShell = &Command{
@@ -24,16 +28,17 @@ var cmdShell = &Command{
`,
}
-var ()
-
func runShell(command *Command, args []string) bool {
util.LoadConfiguration("security", false)
shellOptions.GrpcDialOption = security.LoadClientTLS(viper.Sub("grpc"), "client")
- shellOptions.FilerHost = "localhost"
- shellOptions.FilerPort = 8888
- shellOptions.Directory = "/"
+ var filerPwdErr error
+ shellOptions.FilerHost, shellOptions.FilerPort, shellOptions.Directory, filerPwdErr = util.ParseFilerUrl(*shellInitialFilerUrl)
+ if filerPwdErr != nil {
+ fmt.Printf("failed to parse url filer.url=%s : %v\n", *shellInitialFilerUrl, filerPwdErr)
+ return false
+ }
shell.RunShell(shellOptions)
diff --git a/weed/command/volume.go b/weed/command/volume.go
index 3c1aa2b50..3e8341ef8 100644
--- a/weed/command/volume.go
+++ b/weed/command/volume.go
@@ -1,6 +1,7 @@
package command
import (
+ "fmt"
"net/http"
"os"
"runtime"
@@ -10,7 +11,9 @@ import (
"time"
"github.com/chrislusf/seaweedfs/weed/security"
+ "github.com/chrislusf/seaweedfs/weed/util/httpdown"
"github.com/spf13/viper"
+ "google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
@@ -94,7 +97,7 @@ func runVolume(cmd *Command, args []string) bool {
func (v VolumeServerOptions) startVolumeServer(volumeFolders, maxVolumeCounts, volumeWhiteListOption string) {
- //Set multiple folders and each folder's max volume count limit'
+ // Set multiple folders and each folder's max volume count limit'
v.folders = strings.Split(volumeFolders, ",")
maxCountStrings := strings.Split(maxVolumeCounts, ",")
for _, maxString := range maxCountStrings {
@@ -113,7 +116,7 @@ func (v VolumeServerOptions) startVolumeServer(volumeFolders, maxVolumeCounts, v
}
}
- //security related white list configuration
+ // security related white list configuration
if volumeWhiteListOption != "" {
v.whiteList = strings.Split(volumeWhiteListOption, ",")
}
@@ -128,11 +131,10 @@ func (v VolumeServerOptions) startVolumeServer(volumeFolders, maxVolumeCounts, v
if *v.publicUrl == "" {
*v.publicUrl = *v.ip + ":" + strconv.Itoa(*v.publicPort)
}
- isSeperatedPublicPort := *v.publicPort != *v.port
volumeMux := http.NewServeMux()
publicVolumeMux := volumeMux
- if isSeperatedPublicPort {
+ if v.isSeparatedPublicPort() {
publicVolumeMux = http.NewServeMux()
}
@@ -158,51 +160,131 @@ func (v VolumeServerOptions) startVolumeServer(volumeFolders, maxVolumeCounts, v
*v.compactionMBPerSecond,
)
- listeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.port)
- glog.V(0).Infof("Start Seaweed volume server %s at %s", util.VERSION, listeningAddress)
- listener, e := util.NewListener(listeningAddress, time.Duration(*v.idleConnectionTimeout)*time.Second)
- if e != nil {
- glog.Fatalf("Volume server listener error:%v", e)
- }
- if isSeperatedPublicPort {
- publicListeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.publicPort)
- glog.V(0).Infoln("Start Seaweed volume server", util.VERSION, "public at", publicListeningAddress)
- publicListener, e := util.NewListener(publicListeningAddress, time.Duration(*v.idleConnectionTimeout)*time.Second)
- if e != nil {
- glog.Fatalf("Volume server listener error:%v", e)
+ // starting grpc server
+ grpcS := v.startGrpcService(volumeServer)
+
+ // starting public http server
+ var publicHttpDown httpdown.Server
+ if v.isSeparatedPublicPort() {
+ publicHttpDown = v.startPublicHttpService(publicVolumeMux)
+ if nil == publicHttpDown {
+ glog.Fatalf("start public http service failed")
}
- go func() {
- if e := http.Serve(publicListener, publicVolumeMux); e != nil {
- glog.Fatalf("Volume server fail to serve public: %v", e)
- }
- }()
}
+ // starting the cluster http server
+ clusterHttpServer := v.startClusterHttpService(volumeMux)
+
+ stopChain := make(chan struct{})
util.OnInterrupt(func() {
+ fmt.Println("volume server has be killed")
+ var startTime time.Time
+
+ // firstly, stop the public http service to prevent from receiving new user request
+ if nil != publicHttpDown {
+ startTime = time.Now()
+ if err := publicHttpDown.Stop(); err != nil {
+ glog.Warningf("stop the public http server failed, %v", err)
+ }
+ delta := time.Now().Sub(startTime).Nanoseconds() / 1e6
+ glog.V(0).Infof("stop public http server, elapsed %dms", delta)
+ }
+
+ startTime = time.Now()
+ if err := clusterHttpServer.Stop(); err != nil {
+ glog.Warningf("stop the cluster http server failed, %v", err)
+ }
+ delta := time.Now().Sub(startTime).Nanoseconds() / 1e6
+ glog.V(0).Infof("graceful stop cluster http server, elapsed [%d]", delta)
+
+ startTime = time.Now()
+ grpcS.GracefulStop()
+ delta = time.Now().Sub(startTime).Nanoseconds() / 1e6
+ glog.V(0).Infof("graceful stop gRPC, elapsed [%d]", delta)
+
+ startTime = time.Now()
volumeServer.Shutdown()
+ delta = time.Now().Sub(startTime).Nanoseconds() / 1e6
+ glog.V(0).Infof("stop volume server, elapsed [%d]", delta)
+
pprof.StopCPUProfile()
+
+ close(stopChain) // notify exit
})
- // starting grpc server
+ select {
+ case <-stopChain:
+ }
+ glog.Warningf("the volume server exit.")
+}
+
+// check whether configure the public port
+func (v VolumeServerOptions) isSeparatedPublicPort() bool {
+ return *v.publicPort != *v.port
+}
+
+func (v VolumeServerOptions) startGrpcService(vs volume_server_pb.VolumeServerServer) *grpc.Server {
grpcPort := *v.port + 10000
grpcL, err := util.NewListener(*v.bindIp+":"+strconv.Itoa(grpcPort), 0)
if err != nil {
glog.Fatalf("failed to listen on grpc port %d: %v", grpcPort, err)
}
grpcS := util.NewGrpcServer(security.LoadServerTLS(viper.Sub("grpc"), "volume"))
- volume_server_pb.RegisterVolumeServerServer(grpcS, volumeServer)
+ volume_server_pb.RegisterVolumeServerServer(grpcS, vs)
reflection.Register(grpcS)
- go grpcS.Serve(grpcL)
-
- if viper.GetString("https.volume.key") != "" {
- if e := http.ServeTLS(listener, volumeMux,
- viper.GetString("https.volume.cert"), viper.GetString("https.volume.key")); e != nil {
- glog.Fatalf("Volume server fail to serve: %v", e)
+ go func() {
+ if err := grpcS.Serve(grpcL); err != nil {
+ glog.Fatalf("start gRPC service failed, %s", err)
}
- } else {
- if e := http.Serve(listener, volumeMux); e != nil {
- glog.Fatalf("Volume server fail to serve: %v", e)
+ }()
+ return grpcS
+}
+
+func (v VolumeServerOptions) startPublicHttpService(handler http.Handler) httpdown.Server {
+ publicListeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.publicPort)
+ glog.V(0).Infoln("Start Seaweed volume server", util.VERSION, "public at", publicListeningAddress)
+ publicListener, e := util.NewListener(publicListeningAddress, time.Duration(*v.idleConnectionTimeout)*time.Second)
+ if e != nil {
+ glog.Fatalf("Volume server listener error:%v", e)
+ }
+
+ pubHttp := httpdown.HTTP{StopTimeout: 5 * time.Minute, KillTimeout: 5 * time.Minute}
+ publicHttpDown := pubHttp.Serve(&http.Server{Handler: handler}, publicListener)
+ go func() {
+ if err := publicHttpDown.Wait(); err != nil {
+ glog.Errorf("public http down wait failed, %v", err)
}
+ }()
+
+ return publicHttpDown
+}
+
+func (v VolumeServerOptions) startClusterHttpService(handler http.Handler) httpdown.Server {
+ var (
+ certFile, keyFile string
+ )
+ if viper.GetString("https.volume.key") != "" {
+ certFile = viper.GetString("https.volume.cert")
+ keyFile = viper.GetString("https.volume.key")
+ }
+
+ listeningAddress := *v.bindIp + ":" + strconv.Itoa(*v.port)
+ glog.V(0).Infof("Start Seaweed volume server %s at %s", util.VERSION, listeningAddress)
+ listener, e := util.NewListener(listeningAddress, time.Duration(*v.idleConnectionTimeout)*time.Second)
+ if e != nil {
+ glog.Fatalf("Volume server listener error:%v", e)
}
+ httpDown := httpdown.HTTP{
+ KillTimeout: 5 * time.Minute,
+ StopTimeout: 5 * time.Minute,
+ CertFile: certFile,
+ KeyFile: keyFile}
+ clusterHttpServer := httpDown.Serve(&http.Server{Handler: handler}, listener)
+ go func() {
+ if e := clusterHttpServer.Wait(); e != nil {
+ glog.Fatalf("Volume server fail to serve: %v", e)
+ }
+ }()
+ return clusterHttpServer
}
diff --git a/weed/command/weedfuse/README.md b/weed/command/weedfuse/README.md
deleted file mode 100644
index 1a1496bbb..000000000
--- a/weed/command/weedfuse/README.md
+++ /dev/null
@@ -1,84 +0,0 @@
-Mount the SeaweedFS via FUSE
-
-# Mount by fstab
-
-
-```
-$ # on linux
-$ sudo apt-get install fuse
-$ sudo echo 'user_allow_other' >> /etc/fuse.conf
-$ sudo mv weedfuse /sbin/mount.weedfuse
-
-$ # on Mac
-$ sudo mv weedfuse /sbin/mount_weedfuse
-
-```
-
-On both OS X and Linux, you can add one of the entries to your /etc/fstab file like the following:
-
-```
-# mount the whole SeaweedFS
-localhost:8888/ /home/some/mount/folder weedfuse
-
-# mount the SeaweedFS sub folder
-localhost:8888/sub/dir /home/some/mount/folder weedfuse
-
-# mount the SeaweedFS sub folder with some options
-localhost:8888/sub/dir /home/some/mount/folder weedfuse user
-
-```
-
-To verify it can work, try this command
-```
-$ sudo mount -av
-
-...
-
-/home/some/mount/folder : successfully mounted
-
-```
-
-If you see `successfully mounted`, try to access the mounted folder and verify everything works.
-
-
-To debug, run these:
-```
-
-$ weedfuse -foreground localhost:8888/ /home/some/mount/folder
-
-```
-
-
-To unmount the folder:
-```
-
-$ sudo umount /home/some/mount/folder
-
-```
-
-
diff --git a/weed/command/weedfuse/weedfuse.go b/weed/command/weedfuse/weedfuse.go
deleted file mode 100644
index 4c0d12874..000000000
--- a/weed/command/weedfuse/weedfuse.go
+++ /dev/null
@@ -1,109 +0,0 @@
-package main
-
-import (
- "flag"
- "fmt"
- "os"
- "strings"
-
- "github.com/chrislusf/seaweedfs/weed/command"
- "github.com/chrislusf/seaweedfs/weed/glog"
- "github.com/jacobsa/daemonize"
- "github.com/kardianos/osext"
-)
-
-var (
- fuseCommand = flag.NewFlagSet("weedfuse", flag.ContinueOnError)
- options = fuseCommand.String("o", "", "comma separated options rw,uid=xxx,gid=xxx")
- isForeground = fuseCommand.Bool("foreground", false, "starts as a daemon")
-)
-
-func main() {
-
- err := fuseCommand.Parse(os.Args[1:])
- if err != nil {
- glog.Fatal(err)
- }
- fmt.Printf("options: %v\n", *options)
-
- // seems this value is always empty, need to parse it differently
- optionsString := *options
- prev := ""
- for i, arg := range os.Args {
- fmt.Printf("args[%d]: %v\n", i, arg)
- if prev == "-o" {
- optionsString = arg
- }
- prev = arg
- }
-
- device := fuseCommand.Arg(0)
- mountPoint := fuseCommand.Arg(1)
-
- fmt.Printf("source: %v\n", device)
- fmt.Printf("target: %v\n", mountPoint)
-
- nouser := true
- for _, option := range strings.Split(optionsString, ",") {
- fmt.Printf("option: %v\n", option)
- switch option {
- case "user":
- nouser = false
- }
- }
-
- maybeSetupPath()
-
- if !*isForeground {
- startAsDaemon()
- return
- }
-
- parts := strings.SplitN(device, "/", 2)
- filer, filerPath := parts[0], parts[1]
-
- command.RunMount(
- filer, "/"+filerPath, mountPoint, "", "000", "",
- 4, !nouser, 0, 1000000)
-
-}
-
-func maybeSetupPath() {
- // sudo mount -av may not include PATH in some linux, e.g., Ubuntu
- hasPathEnv := false
- for _, e := range os.Environ() {
- if strings.HasPrefix(e, "PATH=") {
- hasPathEnv = true
- }
- fmt.Println(e)
- }
- if !hasPathEnv {
- os.Setenv("PATH", "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin")
- }
-}
-
-func startAsDaemon() {
-
- // adapted from gcsfuse
-
- // Find the executable.
- var path string
- path, err := osext.Executable()
- if err != nil {
- glog.Fatalf("osext.Executable: %v", err)
- }
-
- // Set up arguments. Be sure to use foreground mode.
- args := append([]string{"-foreground"}, os.Args[1:]...)
-
- // Pass along PATH so that the daemon can find fusermount on Linux.
- env := []string{
- fmt.Sprintf("PATH=%s", os.Getenv("PATH")),
- }
-
- err = daemonize.Run(path, args, env, os.Stdout)
- if err != nil {
- glog.Fatalf("daemonize.Run: %v", err)
- }
-
-}
diff --git a/weed/filer2/abstract_sql/abstract_sql_store.go b/weed/filer2/abstract_sql/abstract_sql_store.go
index 3e8554957..d512467c7 100644
--- a/weed/filer2/abstract_sql/abstract_sql_store.go
+++ b/weed/filer2/abstract_sql/abstract_sql_store.go
@@ -10,13 +10,14 @@ import (
)
type AbstractSqlStore struct {
- DB *sql.DB
- SqlInsert string
- SqlUpdate string
- SqlFind string
- SqlDelete string
- SqlListExclusive string
- SqlListInclusive string
+ DB *sql.DB
+ SqlInsert string
+ SqlUpdate string
+ SqlFind string
+ SqlDelete string
+ SqlDeleteFolderChildren string
+ SqlListExclusive string
+ SqlListInclusive string
}
type TxOrDB interface {
@@ -132,6 +133,21 @@ func (store *AbstractSqlStore) DeleteEntry(ctx context.Context, fullpath filer2.
return nil
}
+func (store *AbstractSqlStore) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) error {
+
+ res, err := store.getTxOrDB(ctx).ExecContext(ctx, store.SqlDeleteFolderChildren, hashToLong(string(fullpath)), fullpath)
+ if err != nil {
+ return fmt.Errorf("deleteFolderChildren %s: %s", fullpath, err)
+ }
+
+ _, err = res.RowsAffected()
+ if err != nil {
+ return fmt.Errorf("deleteFolderChildren %s but no rows affected: %s", fullpath, err)
+ }
+
+ return nil
+}
+
func (store *AbstractSqlStore) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool, limit int) (entries []*filer2.Entry, err error) {
sqlText := store.SqlListExclusive
diff --git a/weed/filer2/cassandra/cassandra_store.go b/weed/filer2/cassandra/cassandra_store.go
index 466be5bf3..dcaab8bc4 100644
--- a/weed/filer2/cassandra/cassandra_store.go
+++ b/weed/filer2/cassandra/cassandra_store.go
@@ -112,6 +112,17 @@ func (store *CassandraStore) DeleteEntry(ctx context.Context, fullpath filer2.Fu
return nil
}
+func (store *CassandraStore) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) error {
+
+ if err := store.session.Query(
+ "DELETE FROM filemeta WHERE directory=?",
+ fullpath).Exec(); err != nil {
+ return fmt.Errorf("delete %s : %v", fullpath, err)
+ }
+
+ return nil
+}
+
func (store *CassandraStore) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool,
limit int) (entries []*filer2.Entry, err error) {
diff --git a/weed/filer2/entry.go b/weed/filer2/entry.go
index 3f8a19114..c901927bb 100644
--- a/weed/filer2/entry.go
+++ b/weed/filer2/entry.go
@@ -30,6 +30,7 @@ type Entry struct {
FullPath
Attr
+ Extended map[string][]byte
// the following is for files
Chunks []*filer_pb.FileChunk `json:"chunks,omitempty"`
@@ -56,6 +57,7 @@ func (entry *Entry) ToProtoEntry() *filer_pb.Entry {
IsDirectory: entry.IsDirectory(),
Attributes: EntryAttributeToPb(entry),
Chunks: entry.Chunks,
+ Extended: entry.Extended,
}
}
diff --git a/weed/filer2/entry_codec.go b/weed/filer2/entry_codec.go
index cf4627b74..3a2dc6134 100644
--- a/weed/filer2/entry_codec.go
+++ b/weed/filer2/entry_codec.go
@@ -1,18 +1,21 @@
package filer2
import (
+ "bytes"
+ "fmt"
"os"
"time"
- "fmt"
- "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/golang/protobuf/proto"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
func (entry *Entry) EncodeAttributesAndChunks() ([]byte, error) {
message := &filer_pb.Entry{
Attributes: EntryAttributeToPb(entry),
Chunks: entry.Chunks,
+ Extended: entry.Extended,
}
return proto.Marshal(message)
}
@@ -27,6 +30,8 @@ func (entry *Entry) DecodeAttributesAndChunks(blob []byte) error {
entry.Attr = PbToEntryAttribute(message.Attributes)
+ entry.Extended = message.Extended
+
entry.Chunks = message.Chunks
return nil
@@ -84,6 +89,10 @@ func EqualEntry(a, b *Entry) bool {
return false
}
+ if !eq(a.Extended, b.Extended) {
+ return false
+ }
+
for i := 0; i < len(a.Chunks); i++ {
if !proto.Equal(a.Chunks[i], b.Chunks[i]) {
return false
@@ -91,3 +100,17 @@ func EqualEntry(a, b *Entry) bool {
}
return true
}
+
+func eq(a, b map[string][]byte) bool {
+ if len(a) != len(b) {
+ return false
+ }
+
+ for k, v := range a {
+ if w, ok := b[k]; !ok || !bytes.Equal(v, w) {
+ return false
+ }
+ }
+
+ return true
+}
diff --git a/weed/filer2/etcd/etcd_store.go b/weed/filer2/etcd/etcd_store.go
new file mode 100644
index 000000000..2eb9e3e86
--- /dev/null
+++ b/weed/filer2/etcd/etcd_store.go
@@ -0,0 +1,196 @@
+package etcd
+
+import (
+ "context"
+ "fmt"
+ "strings"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ weed_util "github.com/chrislusf/seaweedfs/weed/util"
+ "go.etcd.io/etcd/clientv3"
+)
+
+const (
+ DIR_FILE_SEPARATOR = byte(0x00)
+)
+
+func init() {
+ filer2.Stores = append(filer2.Stores, &EtcdStore{})
+}
+
+type EtcdStore struct {
+ client *clientv3.Client
+}
+
+func (store *EtcdStore) GetName() string {
+ return "etcd"
+}
+
+func (store *EtcdStore) Initialize(configuration weed_util.Configuration) (err error) {
+ servers := configuration.GetString("servers")
+ if servers == "" {
+ servers = "localhost:2379"
+ }
+
+ timeout := configuration.GetString("timeout")
+ if timeout == "" {
+ timeout = "3s"
+ }
+
+ return store.initialize(servers, timeout)
+}
+
+func (store *EtcdStore) initialize(servers string, timeout string) (err error) {
+ glog.Infof("filer store etcd: %s", servers)
+
+ to, err := time.ParseDuration(timeout)
+ if err != nil {
+ return fmt.Errorf("parse timeout %s: %s", timeout, err)
+ }
+
+ store.client, err = clientv3.New(clientv3.Config{
+ Endpoints: strings.Split(servers, ","),
+ DialTimeout: to,
+ })
+ if err != nil {
+ return fmt.Errorf("connect to etcd %s: %s", servers, err)
+ }
+
+ return
+}
+
+func (store *EtcdStore) BeginTransaction(ctx context.Context) (context.Context, error) {
+ return ctx, nil
+}
+func (store *EtcdStore) CommitTransaction(ctx context.Context) error {
+ return nil
+}
+func (store *EtcdStore) RollbackTransaction(ctx context.Context) error {
+ return nil
+}
+
+func (store *EtcdStore) InsertEntry(ctx context.Context, entry *filer2.Entry) (err error) {
+ key := genKey(entry.DirAndName())
+
+ value, err := entry.EncodeAttributesAndChunks()
+ if err != nil {
+ return fmt.Errorf("encoding %s %+v: %v", entry.FullPath, entry.Attr, err)
+ }
+
+ if _, err := store.client.Put(ctx, string(key), string(value)); err != nil {
+ return fmt.Errorf("persisting %s : %v", entry.FullPath, err)
+ }
+
+ return nil
+}
+
+func (store *EtcdStore) UpdateEntry(ctx context.Context, entry *filer2.Entry) (err error) {
+ return store.InsertEntry(ctx, entry)
+}
+
+func (store *EtcdStore) FindEntry(ctx context.Context, fullpath filer2.FullPath) (entry *filer2.Entry, err error) {
+ key := genKey(fullpath.DirAndName())
+
+ resp, err := store.client.Get(ctx, string(key))
+ if err != nil {
+ return nil, fmt.Errorf("get %s : %v", entry.FullPath, err)
+ }
+
+ if len(resp.Kvs) == 0 {
+ return nil, filer2.ErrNotFound
+ }
+
+ entry = &filer2.Entry{
+ FullPath: fullpath,
+ }
+ err = entry.DecodeAttributesAndChunks(resp.Kvs[0].Value)
+ if err != nil {
+ return entry, fmt.Errorf("decode %s : %v", entry.FullPath, err)
+ }
+
+ return entry, nil
+}
+
+func (store *EtcdStore) DeleteEntry(ctx context.Context, fullpath filer2.FullPath) (err error) {
+ key := genKey(fullpath.DirAndName())
+
+ if _, err := store.client.Delete(ctx, string(key)); err != nil {
+ return fmt.Errorf("delete %s : %v", fullpath, err)
+ }
+
+ return nil
+}
+
+func (store *EtcdStore) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) (err error) {
+ directoryPrefix := genDirectoryKeyPrefix(fullpath, "")
+
+ if _, err := store.client.Delete(ctx, string(directoryPrefix), clientv3.WithPrefix()); err != nil {
+ return fmt.Errorf("deleteFolderChildren %s : %v", fullpath, err)
+ }
+
+ return nil
+}
+
+func (store *EtcdStore) ListDirectoryEntries(
+ ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool, limit int,
+) (entries []*filer2.Entry, err error) {
+ directoryPrefix := genDirectoryKeyPrefix(fullpath, "")
+
+ resp, err := store.client.Get(ctx, string(directoryPrefix),
+ clientv3.WithPrefix(), clientv3.WithSort(clientv3.SortByKey, clientv3.SortDescend))
+ if err != nil {
+ return nil, fmt.Errorf("list %s : %v", fullpath, err)
+ }
+
+ for _, kv := range resp.Kvs {
+ fileName := getNameFromKey(kv.Key)
+ if fileName == "" {
+ continue
+ }
+ if fileName == startFileName && !inclusive {
+ continue
+ }
+ limit--
+ if limit < 0 {
+ break
+ }
+ entry := &filer2.Entry{
+ FullPath: filer2.NewFullPath(string(fullpath), fileName),
+ }
+ if decodeErr := entry.DecodeAttributesAndChunks(kv.Value); decodeErr != nil {
+ err = decodeErr
+ glog.V(0).Infof("list %s : %v", entry.FullPath, err)
+ break
+ }
+ entries = append(entries, entry)
+ }
+
+ return entries, err
+}
+
+func genKey(dirPath, fileName string) (key []byte) {
+ key = []byte(dirPath)
+ key = append(key, DIR_FILE_SEPARATOR)
+ key = append(key, []byte(fileName)...)
+ return key
+}
+
+func genDirectoryKeyPrefix(fullpath filer2.FullPath, startFileName string) (keyPrefix []byte) {
+ keyPrefix = []byte(string(fullpath))
+ keyPrefix = append(keyPrefix, DIR_FILE_SEPARATOR)
+ if len(startFileName) > 0 {
+ keyPrefix = append(keyPrefix, []byte(startFileName)...)
+ }
+ return keyPrefix
+}
+
+func getNameFromKey(key []byte) string {
+ sepIndex := len(key) - 1
+ for sepIndex >= 0 && key[sepIndex] != DIR_FILE_SEPARATOR {
+ sepIndex--
+ }
+
+ return string(key[sepIndex+1:])
+}
diff --git a/weed/filer2/filer.go b/weed/filer2/filer.go
index cf236b74d..b724e20fd 100644
--- a/weed/filer2/filer.go
+++ b/weed/filer2/filer.go
@@ -3,18 +3,21 @@ package filer2
import (
"context"
"fmt"
- "google.golang.org/grpc"
- "math"
"os"
"path/filepath"
"strings"
"time"
+ "google.golang.org/grpc"
+
+ "github.com/karlseguin/ccache"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/wdclient"
- "github.com/karlseguin/ccache"
)
+const PaginationSize = 1024 * 256
+
var (
OS_UID = uint32(os.Getuid())
OS_GID = uint32(os.Getgid())
@@ -32,7 +35,7 @@ func NewFiler(masters []string, grpcDialOption grpc.DialOption) *Filer {
f := &Filer{
directoryCache: ccache.New(ccache.Configure().MaxSize(1000).ItemsToPrune(100)),
MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "filer", masters),
- fileIdDeletionChan: make(chan string, 4096),
+ fileIdDeletionChan: make(chan string, PaginationSize),
GrpcDialOption: grpcDialOption,
}
@@ -203,67 +206,6 @@ func (f *Filer) FindEntry(ctx context.Context, p FullPath) (entry *Entry, err er
return f.store.FindEntry(ctx, p)
}
-func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p FullPath, isRecursive bool, shouldDeleteChunks bool) (err error) {
- entry, err := f.FindEntry(ctx, p)
- if err != nil {
- return err
- }
-
- if entry.IsDirectory() {
- limit := int(1)
- if isRecursive {
- limit = math.MaxInt32
- }
- lastFileName := ""
- includeLastFile := false
- for limit > 0 {
- entries, err := f.ListDirectoryEntries(ctx, p, lastFileName, includeLastFile, 1024)
- if err != nil {
- glog.Errorf("list folder %s: %v", p, err)
- return fmt.Errorf("list folder %s: %v", p, err)
- }
-
- if len(entries) == 0 {
- break
- }
-
- if isRecursive {
- for _, sub := range entries {
- lastFileName = sub.Name()
- err = f.DeleteEntryMetaAndData(ctx, sub.FullPath, isRecursive, shouldDeleteChunks)
- if err != nil {
- return err
- }
- limit--
- if limit <= 0 {
- break
- }
- }
- }
-
- if len(entries) < 1024 {
- break
- }
- }
-
- f.cacheDelDirectory(string(p))
-
- }
-
- if shouldDeleteChunks {
- f.DeleteChunks(p, entry.Chunks)
- }
-
- if p == "/" {
- return nil
- }
- glog.V(3).Infof("deleting entry %v", p)
-
- f.NotifyUpdateEvent(entry, nil, shouldDeleteChunks)
-
- return f.store.DeleteEntry(ctx, p)
-}
-
func (f *Filer) ListDirectoryEntries(ctx context.Context, p FullPath, startFileName string, inclusive bool, limit int) ([]*Entry, error) {
if strings.HasSuffix(string(p), "/") && len(p) > 1 {
p = p[0 : len(p)-1]
diff --git a/weed/filer2/filer_client_util.go b/weed/filer2/filer_client_util.go
index 7e093eea2..1a10f7c20 100644
--- a/weed/filer2/filer_client_util.go
+++ b/weed/filer2/filer_client_util.go
@@ -3,6 +3,8 @@ package filer2
import (
"context"
"fmt"
+ "io"
+ "math"
"strings"
"sync"
@@ -112,47 +114,54 @@ func GetEntry(ctx context.Context, filerClient FilerClient, fullFilePath string)
return err
}
- if resp.Entry != nil {
- entry = resp.Entry
+ if resp.Entry == nil {
+ glog.V(3).Infof("read %s entry: %v", fullFilePath, entry)
+ return nil
}
+ entry = resp.Entry
return nil
})
return
}
-func ReadDirAllEntries(ctx context.Context, filerClient FilerClient, fullDirPath string, fn func(entry *filer_pb.Entry)) (err error) {
+func ReadDirAllEntries(ctx context.Context, filerClient FilerClient, fullDirPath, prefix string, fn func(entry *filer_pb.Entry, isLast bool)) (err error) {
err = filerClient.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
- paginationLimit := 1024
-
lastEntryName := ""
- for {
-
- request := &filer_pb.ListEntriesRequest{
- Directory: fullDirPath,
- StartFromFileName: lastEntryName,
- Limit: uint32(paginationLimit),
- }
+ request := &filer_pb.ListEntriesRequest{
+ Directory: fullDirPath,
+ Prefix: prefix,
+ StartFromFileName: lastEntryName,
+ Limit: math.MaxUint32,
+ }
- glog.V(3).Infof("read directory: %v", request)
- resp, err := client.ListEntries(ctx, request)
- if err != nil {
- return fmt.Errorf("list %s: %v", fullDirPath, err)
- }
+ glog.V(3).Infof("read directory: %v", request)
+ stream, err := client.ListEntries(ctx, request)
+ if err != nil {
+ return fmt.Errorf("list %s: %v", fullDirPath, err)
+ }
- for _, entry := range resp.Entries {
- fn(entry)
- lastEntryName = entry.Name
+ var prevEntry *filer_pb.Entry
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ if prevEntry != nil {
+ fn(prevEntry, true)
+ }
+ break
+ } else {
+ return recvErr
+ }
}
-
- if len(resp.Entries) < paginationLimit {
- break
+ if prevEntry != nil {
+ fn(prevEntry, false)
}
-
+ prevEntry = resp.Entry
}
return nil
diff --git a/weed/filer2/filer_delete_entry.go b/weed/filer2/filer_delete_entry.go
new file mode 100644
index 000000000..75a09e7ef
--- /dev/null
+++ b/weed/filer2/filer_delete_entry.go
@@ -0,0 +1,102 @@
+package filer2
+
+import (
+ "context"
+ "fmt"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+)
+
+func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p FullPath, isRecursive bool, ignoreRecursiveError, shouldDeleteChunks bool) (err error) {
+ if p == "/" {
+ return nil
+ }
+
+ entry, findErr := f.FindEntry(ctx, p)
+ if findErr != nil {
+ return findErr
+ }
+
+ var chunks []*filer_pb.FileChunk
+ chunks = append(chunks, entry.Chunks...)
+ if entry.IsDirectory() {
+ // delete the folder children, not including the folder itself
+ var dirChunks []*filer_pb.FileChunk
+ dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, entry, isRecursive, ignoreRecursiveError, shouldDeleteChunks)
+ if err != nil {
+ return fmt.Errorf("delete directory %s: %v", p, err)
+ }
+ chunks = append(chunks, dirChunks...)
+ f.cacheDelDirectory(string(p))
+ }
+ // delete the file or folder
+ err = f.doDeleteEntryMetaAndData(ctx, entry, shouldDeleteChunks)
+ if err != nil {
+ return fmt.Errorf("delete file %s: %v", p, err)
+ }
+
+ if shouldDeleteChunks {
+ go f.DeleteChunks(chunks)
+ }
+
+ return nil
+}
+
+func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry, isRecursive bool, ignoreRecursiveError, shouldDeleteChunks bool) (chunks []*filer_pb.FileChunk, err error) {
+
+ lastFileName := ""
+ includeLastFile := false
+ for {
+ entries, err := f.ListDirectoryEntries(ctx, entry.FullPath, lastFileName, includeLastFile, PaginationSize)
+ if err != nil {
+ glog.Errorf("list folder %s: %v", entry.FullPath, err)
+ return nil, fmt.Errorf("list folder %s: %v", entry.FullPath, err)
+ }
+ if lastFileName == "" && !isRecursive && len(entries) > 0 {
+ // only for first iteration in the loop
+ return nil, fmt.Errorf("fail to delete non-empty folder: %s", entry.FullPath)
+ }
+
+ for _, sub := range entries {
+ lastFileName = sub.Name()
+ var dirChunks []*filer_pb.FileChunk
+ if sub.IsDirectory() {
+ dirChunks, err = f.doBatchDeleteFolderMetaAndData(ctx, sub, isRecursive, ignoreRecursiveError, shouldDeleteChunks)
+ }
+ if err != nil && !ignoreRecursiveError {
+ return nil, err
+ }
+ if shouldDeleteChunks {
+ chunks = append(chunks, dirChunks...)
+ }
+ }
+
+ if len(entries) < PaginationSize {
+ break
+ }
+ }
+
+ f.cacheDelDirectory(string(entry.FullPath))
+
+ glog.V(3).Infof("deleting directory %v", entry.FullPath)
+
+ if storeDeletionErr := f.store.DeleteFolderChildren(ctx, entry.FullPath); storeDeletionErr != nil {
+ return nil, fmt.Errorf("filer store delete: %v", storeDeletionErr)
+ }
+ f.NotifyUpdateEvent(entry, nil, shouldDeleteChunks)
+
+ return chunks, nil
+}
+
+func (f *Filer) doDeleteEntryMetaAndData(ctx context.Context, entry *Entry, shouldDeleteChunks bool) (err error) {
+
+ glog.V(3).Infof("deleting entry %v", entry.FullPath)
+
+ if storeDeletionErr := f.store.DeleteEntry(ctx, entry.FullPath); storeDeletionErr != nil {
+ return fmt.Errorf("filer store delete: %v", storeDeletionErr)
+ }
+ f.NotifyUpdateEvent(entry, nil, shouldDeleteChunks)
+
+ return nil
+}
diff --git a/weed/filer2/filer_deletion.go b/weed/filer2/filer_deletion.go
index fea93d57f..9937685f7 100644
--- a/weed/filer2/filer_deletion.go
+++ b/weed/filer2/filer_deletion.go
@@ -15,7 +15,7 @@ func (f *Filer) loopProcessingDeletion() {
lookupFunc := func(vids []string) (map[string]operation.LookupResult, error) {
m := make(map[string]operation.LookupResult)
for _, vid := range vids {
- locs := f.MasterClient.GetVidLocations(vid)
+ locs, _ := f.MasterClient.GetVidLocations(vid)
var locations []operation.Location
for _, loc := range locs {
locations = append(locations, operation.Location{
@@ -51,9 +51,8 @@ func (f *Filer) loopProcessingDeletion() {
}
}
-func (f *Filer) DeleteChunks(fullpath FullPath, chunks []*filer_pb.FileChunk) {
+func (f *Filer) DeleteChunks(chunks []*filer_pb.FileChunk) {
for _, chunk := range chunks {
- glog.V(3).Infof("deleting %s chunk %s", fullpath, chunk.String())
f.fileIdDeletionChan <- chunk.GetFileIdString()
}
}
@@ -70,7 +69,7 @@ func (f *Filer) deleteChunksIfNotNew(oldEntry, newEntry *Entry) {
return
}
if newEntry == nil {
- f.DeleteChunks(oldEntry.FullPath, oldEntry.Chunks)
+ f.DeleteChunks(oldEntry.Chunks)
}
var toDelete []*filer_pb.FileChunk
@@ -84,5 +83,5 @@ func (f *Filer) deleteChunksIfNotNew(oldEntry, newEntry *Entry) {
toDelete = append(toDelete, oldChunk)
}
}
- f.DeleteChunks(oldEntry.FullPath, toDelete)
+ f.DeleteChunks(toDelete)
}
diff --git a/weed/filer2/filerstore.go b/weed/filer2/filerstore.go
index 231c7fc68..0bb0bd611 100644
--- a/weed/filer2/filerstore.go
+++ b/weed/filer2/filerstore.go
@@ -20,6 +20,7 @@ type FilerStore interface {
// err == filer2.ErrNotFound if not found
FindEntry(context.Context, FullPath) (entry *Entry, err error)
DeleteEntry(context.Context, FullPath) (err error)
+ DeleteFolderChildren(context.Context, FullPath) (err error)
ListDirectoryEntries(ctx context.Context, dirPath FullPath, startFileName string, includeStartFile bool, limit int) ([]*Entry, error)
BeginTransaction(ctx context.Context) (context.Context, error)
@@ -34,6 +35,9 @@ type FilerStoreWrapper struct {
}
func NewFilerStoreWrapper(store FilerStore) *FilerStoreWrapper {
+ if innerStore, ok := store.(*FilerStoreWrapper); ok {
+ return innerStore
+ }
return &FilerStoreWrapper{
actualStore: store,
}
@@ -94,6 +98,16 @@ func (fsw *FilerStoreWrapper) DeleteEntry(ctx context.Context, fp FullPath) (err
return fsw.actualStore.DeleteEntry(ctx, fp)
}
+func (fsw *FilerStoreWrapper) DeleteFolderChildren(ctx context.Context, fp FullPath) (err error) {
+ stats.FilerStoreCounter.WithLabelValues(fsw.actualStore.GetName(), "deleteFolderChildren").Inc()
+ start := time.Now()
+ defer func() {
+ stats.FilerStoreHistogram.WithLabelValues(fsw.actualStore.GetName(), "deleteFolderChildren").Observe(time.Since(start).Seconds())
+ }()
+
+ return fsw.actualStore.DeleteFolderChildren(ctx, fp)
+}
+
func (fsw *FilerStoreWrapper) ListDirectoryEntries(ctx context.Context, dirPath FullPath, startFileName string, includeStartFile bool, limit int) ([]*Entry, error) {
stats.FilerStoreCounter.WithLabelValues(fsw.actualStore.GetName(), "list").Inc()
start := time.Now()
diff --git a/weed/filer2/leveldb/leveldb_store.go b/weed/filer2/leveldb/leveldb_store.go
index d00eba859..4952b3b3a 100644
--- a/weed/filer2/leveldb/leveldb_store.go
+++ b/weed/filer2/leveldb/leveldb_store.go
@@ -5,12 +5,13 @@ import (
"context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "github.com/chrislusf/seaweedfs/weed/glog"
- weed_util "github.com/chrislusf/seaweedfs/weed/util"
"github.com/syndtr/goleveldb/leveldb"
"github.com/syndtr/goleveldb/leveldb/opt"
leveldb_util "github.com/syndtr/goleveldb/leveldb/util"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ weed_util "github.com/chrislusf/seaweedfs/weed/util"
)
const (
@@ -123,6 +124,34 @@ func (store *LevelDBStore) DeleteEntry(ctx context.Context, fullpath filer2.Full
return nil
}
+func (store *LevelDBStore) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) (err error) {
+
+ batch := new(leveldb.Batch)
+
+ directoryPrefix := genDirectoryKeyPrefix(fullpath, "")
+ iter := store.db.NewIterator(&leveldb_util.Range{Start: directoryPrefix}, nil)
+ for iter.Next() {
+ key := iter.Key()
+ if !bytes.HasPrefix(key, directoryPrefix) {
+ break
+ }
+ fileName := getNameFromKey(key)
+ if fileName == "" {
+ continue
+ }
+ batch.Delete([]byte(genKey(string(fullpath), fileName)))
+ }
+ iter.Release()
+
+ err = store.db.Write(batch, nil)
+
+ if err != nil {
+ return fmt.Errorf("delete %s : %v", fullpath, err)
+ }
+
+ return nil
+}
+
func (store *LevelDBStore) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool,
limit int) (entries []*filer2.Entry, err error) {
diff --git a/weed/filer2/leveldb2/leveldb2_store.go b/weed/filer2/leveldb2/leveldb2_store.go
index bce81e357..8a16822ab 100644
--- a/weed/filer2/leveldb2/leveldb2_store.go
+++ b/weed/filer2/leveldb2/leveldb2_store.go
@@ -8,12 +8,13 @@ import (
"io"
"os"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "github.com/chrislusf/seaweedfs/weed/glog"
- weed_util "github.com/chrislusf/seaweedfs/weed/util"
"github.com/syndtr/goleveldb/leveldb"
"github.com/syndtr/goleveldb/leveldb/opt"
leveldb_util "github.com/syndtr/goleveldb/leveldb/util"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ weed_util "github.com/chrislusf/seaweedfs/weed/util"
)
func init() {
@@ -21,7 +22,7 @@ func init() {
}
type LevelDB2Store struct {
- dbs []*leveldb.DB
+ dbs []*leveldb.DB
dbCount int
}
@@ -46,7 +47,7 @@ func (store *LevelDB2Store) initialize(dir string, dbCount int) (err error) {
CompactionTableSizeMultiplier: 4,
}
- for d := 0 ; d < dbCount; d++ {
+ for d := 0; d < dbCount; d++ {
dbFolder := fmt.Sprintf("%s/%02d", dir, d)
os.MkdirAll(dbFolder, 0755)
db, dbErr := leveldb.OpenFile(dbFolder, opts)
@@ -134,6 +135,34 @@ func (store *LevelDB2Store) DeleteEntry(ctx context.Context, fullpath filer2.Ful
return nil
}
+func (store *LevelDB2Store) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) (err error) {
+ directoryPrefix, partitionId := genDirectoryKeyPrefix(fullpath, "", store.dbCount)
+
+ batch := new(leveldb.Batch)
+
+ iter := store.dbs[partitionId].NewIterator(&leveldb_util.Range{Start: directoryPrefix}, nil)
+ for iter.Next() {
+ key := iter.Key()
+ if !bytes.HasPrefix(key, directoryPrefix) {
+ break
+ }
+ fileName := getNameFromKey(key)
+ if fileName == "" {
+ continue
+ }
+ batch.Delete(append(directoryPrefix, []byte(fileName)...))
+ }
+ iter.Release()
+
+ err = store.dbs[partitionId].Write(batch, nil)
+
+ if err != nil {
+ return fmt.Errorf("delete %s : %v", fullpath, err)
+ }
+
+ return nil
+}
+
func (store *LevelDB2Store) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool,
limit int) (entries []*filer2.Entry, err error) {
@@ -204,5 +233,5 @@ func hashToBytes(dir string, dbCount int) ([]byte, int) {
x := b[len(b)-1]
- return b, int(x)%dbCount
+ return b, int(x) % dbCount
}
diff --git a/weed/filer2/leveldb2/leveldb2_store_test.go b/weed/filer2/leveldb2/leveldb2_store_test.go
index a16803ca1..e28ef7dac 100644
--- a/weed/filer2/leveldb2/leveldb2_store_test.go
+++ b/weed/filer2/leveldb2/leveldb2_store_test.go
@@ -13,7 +13,7 @@ func TestCreateAndFind(t *testing.T) {
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test")
defer os.RemoveAll(dir)
store := &LevelDB2Store{}
- store.initialize(dir,2)
+ store.initialize(dir, 2)
filer.SetStore(store)
filer.DisableDirectoryCache()
@@ -68,7 +68,7 @@ func TestEmptyRoot(t *testing.T) {
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test2")
defer os.RemoveAll(dir)
store := &LevelDB2Store{}
- store.initialize(dir,2)
+ store.initialize(dir, 2)
filer.SetStore(store)
filer.DisableDirectoryCache()
diff --git a/weed/filer2/memdb/memdb_store.go b/weed/filer2/memdb/memdb_store.go
deleted file mode 100644
index 9c10a5472..000000000
--- a/weed/filer2/memdb/memdb_store.go
+++ /dev/null
@@ -1,132 +0,0 @@
-package memdb
-
-import (
- "context"
- "fmt"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "github.com/chrislusf/seaweedfs/weed/util"
- "github.com/google/btree"
- "strings"
- "sync"
-)
-
-func init() {
- filer2.Stores = append(filer2.Stores, &MemDbStore{})
-}
-
-type MemDbStore struct {
- tree *btree.BTree
- treeLock sync.Mutex
-}
-
-type entryItem struct {
- *filer2.Entry
-}
-
-func (a entryItem) Less(b btree.Item) bool {
- return strings.Compare(string(a.FullPath), string(b.(entryItem).FullPath)) < 0
-}
-
-func (store *MemDbStore) GetName() string {
- return "memory"
-}
-
-func (store *MemDbStore) Initialize(configuration util.Configuration) (err error) {
- store.tree = btree.New(8)
- return nil
-}
-
-func (store *MemDbStore) BeginTransaction(ctx context.Context) (context.Context, error) {
- return ctx, nil
-}
-func (store *MemDbStore) CommitTransaction(ctx context.Context) error {
- return nil
-}
-func (store *MemDbStore) RollbackTransaction(ctx context.Context) error {
- return nil
-}
-
-func (store *MemDbStore) InsertEntry(ctx context.Context, entry *filer2.Entry) (err error) {
- // println("inserting", entry.FullPath)
- store.treeLock.Lock()
- store.tree.ReplaceOrInsert(entryItem{entry})
- store.treeLock.Unlock()
- return nil
-}
-
-func (store *MemDbStore) UpdateEntry(ctx context.Context, entry *filer2.Entry) (err error) {
- if _, err = store.FindEntry(ctx, entry.FullPath); err != nil {
- return fmt.Errorf("no such file %s : %v", entry.FullPath, err)
- }
- store.treeLock.Lock()
- store.tree.ReplaceOrInsert(entryItem{entry})
- store.treeLock.Unlock()
- return nil
-}
-
-func (store *MemDbStore) FindEntry(ctx context.Context, fullpath filer2.FullPath) (entry *filer2.Entry, err error) {
- item := store.tree.Get(entryItem{&filer2.Entry{FullPath: fullpath}})
- if item == nil {
- return nil, filer2.ErrNotFound
- }
- entry = item.(entryItem).Entry
- return entry, nil
-}
-
-func (store *MemDbStore) DeleteEntry(ctx context.Context, fullpath filer2.FullPath) (err error) {
- store.treeLock.Lock()
- store.tree.Delete(entryItem{&filer2.Entry{FullPath: fullpath}})
- store.treeLock.Unlock()
- return nil
-}
-
-func (store *MemDbStore) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool, limit int) (entries []*filer2.Entry, err error) {
-
- startFrom := string(fullpath)
- if startFileName != "" {
- startFrom = startFrom + "/" + startFileName
- }
-
- store.tree.AscendGreaterOrEqual(entryItem{&filer2.Entry{FullPath: filer2.FullPath(startFrom)}},
- func(item btree.Item) bool {
- if limit <= 0 {
- return false
- }
- entry := item.(entryItem).Entry
- // println("checking", entry.FullPath)
-
- if entry.FullPath == fullpath {
- // skipping the current directory
- // println("skipping the folder", entry.FullPath)
- return true
- }
-
- dir, name := entry.FullPath.DirAndName()
- if name == startFileName {
- if inclusive {
- limit--
- entries = append(entries, entry)
- }
- return true
- }
-
- // only iterate the same prefix
- if !strings.HasPrefix(string(entry.FullPath), string(fullpath)) {
- // println("breaking from", entry.FullPath)
- return false
- }
-
- if dir != string(fullpath) {
- // this could be items in deeper directories
- // println("skipping deeper folder", entry.FullPath)
- return true
- }
- // now process the directory items
- // println("adding entry", entry.FullPath)
- limit--
- entries = append(entries, entry)
- return true
- },
- )
- return entries, nil
-}
diff --git a/weed/filer2/memdb/memdb_store_test.go b/weed/filer2/memdb/memdb_store_test.go
deleted file mode 100644
index d823c5177..000000000
--- a/weed/filer2/memdb/memdb_store_test.go
+++ /dev/null
@@ -1,149 +0,0 @@
-package memdb
-
-import (
- "context"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "testing"
-)
-
-func TestCreateAndFind(t *testing.T) {
- filer := filer2.NewFiler(nil, nil)
- store := &MemDbStore{}
- store.Initialize(nil)
- filer.SetStore(store)
- filer.DisableDirectoryCache()
-
- ctx := context.Background()
-
- fullpath := filer2.FullPath("/home/chris/this/is/one/file1.jpg")
-
- entry1 := &filer2.Entry{
- FullPath: fullpath,
- Attr: filer2.Attr{
- Mode: 0440,
- Uid: 1234,
- Gid: 5678,
- },
- }
-
- if err := filer.CreateEntry(ctx, entry1); err != nil {
- t.Errorf("create entry %v: %v", entry1.FullPath, err)
- return
- }
-
- entry, err := filer.FindEntry(ctx, fullpath)
-
- if err != nil {
- t.Errorf("find entry: %v", err)
- return
- }
-
- if entry.FullPath != entry1.FullPath {
- t.Errorf("find wrong entry: %v", entry.FullPath)
- return
- }
-
-}
-
-func TestCreateFileAndList(t *testing.T) {
- filer := filer2.NewFiler(nil, nil)
- store := &MemDbStore{}
- store.Initialize(nil)
- filer.SetStore(store)
- filer.DisableDirectoryCache()
-
- ctx := context.Background()
-
- entry1 := &filer2.Entry{
- FullPath: filer2.FullPath("/home/chris/this/is/one/file1.jpg"),
- Attr: filer2.Attr{
- Mode: 0440,
- Uid: 1234,
- Gid: 5678,
- },
- }
-
- entry2 := &filer2.Entry{
- FullPath: filer2.FullPath("/home/chris/this/is/one/file2.jpg"),
- Attr: filer2.Attr{
- Mode: 0440,
- Uid: 1234,
- Gid: 5678,
- },
- }
-
- filer.CreateEntry(ctx, entry1)
- filer.CreateEntry(ctx, entry2)
-
- // checking the 2 files
- entries, err := filer.ListDirectoryEntries(ctx, filer2.FullPath("/home/chris/this/is/one/"), "", false, 100)
-
- if err != nil {
- t.Errorf("list entries: %v", err)
- return
- }
-
- if len(entries) != 2 {
- t.Errorf("list entries count: %v", len(entries))
- return
- }
-
- if entries[0].FullPath != entry1.FullPath {
- t.Errorf("find wrong entry 1: %v", entries[0].FullPath)
- return
- }
-
- if entries[1].FullPath != entry2.FullPath {
- t.Errorf("find wrong entry 2: %v", entries[1].FullPath)
- return
- }
-
- // checking the offset
- entries, err = filer.ListDirectoryEntries(ctx, filer2.FullPath("/home/chris/this/is/one/"), "file1.jpg", false, 100)
- if len(entries) != 1 {
- t.Errorf("list entries count: %v", len(entries))
- return
- }
-
- // checking one upper directory
- entries, _ = filer.ListDirectoryEntries(ctx, filer2.FullPath("/home/chris/this/is"), "", false, 100)
- if len(entries) != 1 {
- t.Errorf("list entries count: %v", len(entries))
- return
- }
-
- // checking root directory
- entries, _ = filer.ListDirectoryEntries(ctx, filer2.FullPath("/"), "", false, 100)
- if len(entries) != 1 {
- t.Errorf("list entries count: %v", len(entries))
- return
- }
-
- // add file3
- file3Path := filer2.FullPath("/home/chris/this/is/file3.jpg")
- entry3 := &filer2.Entry{
- FullPath: file3Path,
- Attr: filer2.Attr{
- Mode: 0440,
- Uid: 1234,
- Gid: 5678,
- },
- }
- filer.CreateEntry(ctx, entry3)
-
- // checking one upper directory
- entries, _ = filer.ListDirectoryEntries(ctx, filer2.FullPath("/home/chris/this/is"), "", false, 100)
- if len(entries) != 2 {
- t.Errorf("list entries count: %v", len(entries))
- return
- }
-
- // delete file and count
- filer.DeleteEntryMetaAndData(ctx, file3Path, false, false)
- entries, _ = filer.ListDirectoryEntries(ctx, filer2.FullPath("/home/chris/this/is"), "", false, 100)
- if len(entries) != 1 {
- t.Errorf("list entries count: %v", len(entries))
- return
- }
-
-}
diff --git a/weed/filer2/mysql/mysql_store.go b/weed/filer2/mysql/mysql_store.go
index e18299bd2..d1b06ece5 100644
--- a/weed/filer2/mysql/mysql_store.go
+++ b/weed/filer2/mysql/mysql_store.go
@@ -35,19 +35,26 @@ func (store *MysqlStore) Initialize(configuration util.Configuration) (err error
configuration.GetString("database"),
configuration.GetInt("connection_max_idle"),
configuration.GetInt("connection_max_open"),
+ configuration.GetBool("interpolateParams"),
)
}
-func (store *MysqlStore) initialize(user, password, hostname string, port int, database string, maxIdle, maxOpen int) (err error) {
+func (store *MysqlStore) initialize(user, password, hostname string, port int, database string, maxIdle, maxOpen int,
+ interpolateParams bool) (err error) {
store.SqlInsert = "INSERT INTO filemeta (dirhash,name,directory,meta) VALUES(?,?,?,?)"
store.SqlUpdate = "UPDATE filemeta SET meta=? WHERE dirhash=? AND name=? AND directory=?"
store.SqlFind = "SELECT meta FROM filemeta WHERE dirhash=? AND name=? AND directory=?"
store.SqlDelete = "DELETE FROM filemeta WHERE dirhash=? AND name=? AND directory=?"
+ store.SqlDeleteFolderChildren = "DELETE FROM filemeta WHERE dirhash=? AND directory=?"
store.SqlListExclusive = "SELECT NAME, meta FROM filemeta WHERE dirhash=? AND name>? AND directory=? ORDER BY NAME ASC LIMIT ?"
store.SqlListInclusive = "SELECT NAME, meta FROM filemeta WHERE dirhash=? AND name>=? AND directory=? ORDER BY NAME ASC LIMIT ?"
sqlUrl := fmt.Sprintf(CONNECTION_URL_PATTERN, user, password, hostname, port, database)
+ if interpolateParams {
+ sqlUrl += "&interpolateParams=true"
+ }
+
var dbErr error
store.DB, dbErr = sql.Open("mysql", sqlUrl)
if dbErr != nil {
diff --git a/weed/filer2/postgres/postgres_store.go b/weed/filer2/postgres/postgres_store.go
index ffd3d1e01..3ec000fe0 100644
--- a/weed/filer2/postgres/postgres_store.go
+++ b/weed/filer2/postgres/postgres_store.go
@@ -45,6 +45,7 @@ func (store *PostgresStore) initialize(user, password, hostname string, port int
store.SqlUpdate = "UPDATE filemeta SET meta=$1 WHERE dirhash=$2 AND name=$3 AND directory=$4"
store.SqlFind = "SELECT meta FROM filemeta WHERE dirhash=$1 AND name=$2 AND directory=$3"
store.SqlDelete = "DELETE FROM filemeta WHERE dirhash=$1 AND name=$2 AND directory=$3"
+ store.SqlDeleteFolderChildren = "DELETE FROM filemeta WHERE dirhash=$1 AND directory=$2"
store.SqlListExclusive = "SELECT NAME, meta FROM filemeta WHERE dirhash=$1 AND name>$2 AND directory=$3 ORDER BY NAME ASC LIMIT $4"
store.SqlListInclusive = "SELECT NAME, meta FROM filemeta WHERE dirhash=$1 AND name>=$2 AND directory=$3 ORDER BY NAME ASC LIMIT $4"
diff --git a/weed/filer2/redis/redis_cluster_store.go b/weed/filer2/redis/redis_cluster_store.go
index 11c315391..f1ad4b35c 100644
--- a/weed/filer2/redis/redis_cluster_store.go
+++ b/weed/filer2/redis/redis_cluster_store.go
@@ -19,16 +19,24 @@ func (store *RedisClusterStore) GetName() string {
}
func (store *RedisClusterStore) Initialize(configuration util.Configuration) (err error) {
+
+ configuration.SetDefault("useReadOnly", true)
+ configuration.SetDefault("routeByLatency", true)
+
return store.initialize(
configuration.GetStringSlice("addresses"),
configuration.GetString("password"),
+ configuration.GetBool("useReadOnly"),
+ configuration.GetBool("routeByLatency"),
)
}
-func (store *RedisClusterStore) initialize(addresses []string, password string) (err error) {
+func (store *RedisClusterStore) initialize(addresses []string, password string, readOnly, routeByLatency bool) (err error) {
store.Client = redis.NewClusterClient(&redis.ClusterOptions{
- Addrs: addresses,
- Password: password,
+ Addrs: addresses,
+ Password: password,
+ ReadOnly: readOnly,
+ RouteByLatency: routeByLatency,
})
return
}
diff --git a/weed/filer2/redis/universal_redis_store.go b/weed/filer2/redis/universal_redis_store.go
index ce41d4d70..62257e91e 100644
--- a/weed/filer2/redis/universal_redis_store.go
+++ b/weed/filer2/redis/universal_redis_store.go
@@ -99,6 +99,24 @@ func (store *UniversalRedisStore) DeleteEntry(ctx context.Context, fullpath file
return nil
}
+func (store *UniversalRedisStore) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) (err error) {
+
+ members, err := store.Client.SMembers(genDirectoryListKey(string(fullpath))).Result()
+ if err != nil {
+ return fmt.Errorf("delete folder %s : %v", fullpath, err)
+ }
+
+ for _, fileName := range members {
+ path := filer2.NewFullPath(string(fullpath), fileName)
+ _, err = store.Client.Del(string(path)).Result()
+ if err != nil {
+ return fmt.Errorf("delete %s in parent dir: %v", fullpath, err)
+ }
+ }
+
+ return nil
+}
+
func (store *UniversalRedisStore) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool,
limit int) (entries []*filer2.Entry, err error) {
diff --git a/weed/filer2/tikv/tikv_store.go b/weed/filer2/tikv/tikv_store.go
new file mode 100644
index 000000000..4eb8cb90d
--- /dev/null
+++ b/weed/filer2/tikv/tikv_store.go
@@ -0,0 +1,251 @@
+// +build !386
+// +build !arm
+
+package tikv
+
+import (
+ "bytes"
+ "context"
+ "crypto/md5"
+ "fmt"
+ "io"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ weed_util "github.com/chrislusf/seaweedfs/weed/util"
+
+ "github.com/pingcap/tidb/kv"
+ "github.com/pingcap/tidb/store/tikv"
+)
+
+func init() {
+ filer2.Stores = append(filer2.Stores, &TikvStore{})
+}
+
+type TikvStore struct {
+ store kv.Storage
+}
+
+func (store *TikvStore) GetName() string {
+ return "tikv"
+}
+
+func (store *TikvStore) Initialize(configuration weed_util.Configuration) (err error) {
+ pdAddr := configuration.GetString("pdAddress")
+ return store.initialize(pdAddr)
+}
+
+func (store *TikvStore) initialize(pdAddr string) (err error) {
+ glog.Infof("filer store tikv pd address: %s", pdAddr)
+
+ driver := tikv.Driver{}
+
+ store.store, err = driver.Open(fmt.Sprintf("tikv://%s", pdAddr))
+
+ if err != nil {
+ return fmt.Errorf("open tikv %s : %v", pdAddr, err)
+ }
+
+ return
+}
+
+func (store *TikvStore) BeginTransaction(ctx context.Context) (context.Context, error) {
+ tx, err := store.store.Begin()
+ if err != nil {
+ return ctx, err
+ }
+ return context.WithValue(ctx, "tx", tx), nil
+}
+func (store *TikvStore) CommitTransaction(ctx context.Context) error {
+ tx, ok := ctx.Value("tx").(kv.Transaction)
+ if ok {
+ return tx.Commit(ctx)
+ }
+ return nil
+}
+func (store *TikvStore) RollbackTransaction(ctx context.Context) error {
+ tx, ok := ctx.Value("tx").(kv.Transaction)
+ if ok {
+ return tx.Rollback()
+ }
+ return nil
+}
+
+func (store *TikvStore) getTx(ctx context.Context) kv.Transaction {
+ if tx, ok := ctx.Value("tx").(kv.Transaction); ok {
+ return tx
+ }
+ return nil
+}
+
+func (store *TikvStore) InsertEntry(ctx context.Context, entry *filer2.Entry) (err error) {
+ dir, name := entry.DirAndName()
+ key := genKey(dir, name)
+
+ value, err := entry.EncodeAttributesAndChunks()
+ if err != nil {
+ return fmt.Errorf("encoding %s %+v: %v", entry.FullPath, entry.Attr, err)
+ }
+
+ err = store.getTx(ctx).Set(key, value)
+
+ if err != nil {
+ return fmt.Errorf("persisting %s : %v", entry.FullPath, err)
+ }
+
+ // println("saved", entry.FullPath, "chunks", len(entry.Chunks))
+
+ return nil
+}
+
+func (store *TikvStore) UpdateEntry(ctx context.Context, entry *filer2.Entry) (err error) {
+
+ return store.InsertEntry(ctx, entry)
+}
+
+func (store *TikvStore) FindEntry(ctx context.Context, fullpath filer2.FullPath) (entry *filer2.Entry, err error) {
+ dir, name := fullpath.DirAndName()
+ key := genKey(dir, name)
+
+ data, err := store.getTx(ctx).Get(ctx, key)
+
+ if err == kv.ErrNotExist {
+ return nil, filer2.ErrNotFound
+ }
+ if err != nil {
+ return nil, fmt.Errorf("get %s : %v", entry.FullPath, err)
+ }
+
+ entry = &filer2.Entry{
+ FullPath: fullpath,
+ }
+ err = entry.DecodeAttributesAndChunks(data)
+ if err != nil {
+ return entry, fmt.Errorf("decode %s : %v", entry.FullPath, err)
+ }
+
+ // println("read", entry.FullPath, "chunks", len(entry.Chunks), "data", len(data), string(data))
+
+ return entry, nil
+}
+
+func (store *TikvStore) DeleteEntry(ctx context.Context, fullpath filer2.FullPath) (err error) {
+ dir, name := fullpath.DirAndName()
+ key := genKey(dir, name)
+
+ err = store.getTx(ctx).Delete(key)
+ if err != nil {
+ return fmt.Errorf("delete %s : %v", fullpath, err)
+ }
+
+ return nil
+}
+
+func (store *TikvStore) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) (err error) {
+
+ directoryPrefix := genDirectoryKeyPrefix(fullpath, "")
+
+ tx := store.getTx(ctx)
+
+ iter, err := tx.Iter(directoryPrefix, nil)
+ if err != nil {
+ return fmt.Errorf("deleteFolderChildren %s: %v", fullpath, err)
+ }
+ defer iter.Close()
+ for iter.Valid() {
+ key := iter.Key()
+ if !bytes.HasPrefix(key, directoryPrefix) {
+ break
+ }
+ fileName := getNameFromKey(key)
+ if fileName == "" {
+ iter.Next()
+ continue
+ }
+
+ if err = tx.Delete(genKey(string(fullpath), fileName)); err != nil {
+ return fmt.Errorf("delete %s : %v", fullpath, err)
+ }
+
+ iter.Next()
+ }
+
+ return nil
+}
+
+func (store *TikvStore) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool,
+ limit int) (entries []*filer2.Entry, err error) {
+
+ directoryPrefix := genDirectoryKeyPrefix(fullpath, "")
+ lastFileStart := genDirectoryKeyPrefix(fullpath, startFileName)
+
+ iter, err := store.getTx(ctx).Iter(lastFileStart, nil)
+ if err != nil {
+ return nil, fmt.Errorf("list %s: %v", fullpath, err)
+ }
+ defer iter.Close()
+ for iter.Valid() {
+ key := iter.Key()
+ if !bytes.HasPrefix(key, directoryPrefix) {
+ break
+ }
+ fileName := getNameFromKey(key)
+ if fileName == "" {
+ iter.Next()
+ continue
+ }
+ if fileName == startFileName && !inclusive {
+ iter.Next()
+ continue
+ }
+ limit--
+ if limit < 0 {
+ break
+ }
+ entry := &filer2.Entry{
+ FullPath: filer2.NewFullPath(string(fullpath), fileName),
+ }
+
+ // println("list", entry.FullPath, "chunks", len(entry.Chunks))
+
+ if decodeErr := entry.DecodeAttributesAndChunks(iter.Value()); decodeErr != nil {
+ err = decodeErr
+ glog.V(0).Infof("list %s : %v", entry.FullPath, err)
+ break
+ }
+ entries = append(entries, entry)
+ iter.Next()
+ }
+
+ return entries, err
+}
+
+func genKey(dirPath, fileName string) (key []byte) {
+ key = hashToBytes(dirPath)
+ key = append(key, []byte(fileName)...)
+ return key
+}
+
+func genDirectoryKeyPrefix(fullpath filer2.FullPath, startFileName string) (keyPrefix []byte) {
+ keyPrefix = hashToBytes(string(fullpath))
+ if len(startFileName) > 0 {
+ keyPrefix = append(keyPrefix, []byte(startFileName)...)
+ }
+ return keyPrefix
+}
+
+func getNameFromKey(key []byte) string {
+
+ return string(key[md5.Size:])
+
+}
+
+// hash directory
+func hashToBytes(dir string) []byte {
+ h := md5.New()
+ io.WriteString(h, dir)
+
+ b := h.Sum(nil)
+
+ return b
+}
diff --git a/weed/filer2/tikv/tikv_store_unsupported.go b/weed/filer2/tikv/tikv_store_unsupported.go
new file mode 100644
index 000000000..36de2d974
--- /dev/null
+++ b/weed/filer2/tikv/tikv_store_unsupported.go
@@ -0,0 +1,65 @@
+// +build 386 arm
+
+package tikv
+
+import (
+ "context"
+ "fmt"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ weed_util "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+func init() {
+ filer2.Stores = append(filer2.Stores, &TikvStore{})
+}
+
+type TikvStore struct {
+}
+
+func (store *TikvStore) GetName() string {
+ return "tikv"
+}
+
+func (store *TikvStore) Initialize(configuration weed_util.Configuration) (err error) {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) initialize(pdAddr string) (err error) {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) BeginTransaction(ctx context.Context) (context.Context, error) {
+ return nil, fmt.Errorf("not implemented for 32 bit computers")
+}
+func (store *TikvStore) CommitTransaction(ctx context.Context) error {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+func (store *TikvStore) RollbackTransaction(ctx context.Context) error {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) InsertEntry(ctx context.Context, entry *filer2.Entry) (err error) {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) UpdateEntry(ctx context.Context, entry *filer2.Entry) (err error) {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) FindEntry(ctx context.Context, fullpath filer2.FullPath) (entry *filer2.Entry, err error) {
+ return nil, fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) DeleteEntry(ctx context.Context, fullpath filer2.FullPath) (err error) {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) DeleteFolderChildren(ctx context.Context, fullpath filer2.FullPath) (err error) {
+ return fmt.Errorf("not implemented for 32 bit computers")
+}
+
+func (store *TikvStore) ListDirectoryEntries(ctx context.Context, fullpath filer2.FullPath, startFileName string, inclusive bool,
+ limit int) (entries []*filer2.Entry, err error) {
+ return nil, fmt.Errorf("not implemented for 32 bit computers")
+}
diff --git a/weed/filesys/dir.go b/weed/filesys/dir.go
index 0e9e92e16..7b24a1ec5 100644
--- a/weed/filesys/dir.go
+++ b/weed/filesys/dir.go
@@ -14,9 +14,9 @@ import (
)
type Dir struct {
- Path string
- wfs *WFS
- attributes *filer_pb.FuseAttributes
+ Path string
+ wfs *WFS
+ entry *filer_pb.Entry
}
var _ = fs.Node(&Dir{})
@@ -27,9 +27,15 @@ var _ = fs.HandleReadDirAller(&Dir{})
var _ = fs.NodeRemover(&Dir{})
var _ = fs.NodeRenamer(&Dir{})
var _ = fs.NodeSetattrer(&Dir{})
+var _ = fs.NodeGetxattrer(&Dir{})
+var _ = fs.NodeSetxattrer(&Dir{})
+var _ = fs.NodeRemovexattrer(&Dir{})
+var _ = fs.NodeListxattrer(&Dir{})
func (dir *Dir) Attr(ctx context.Context, attr *fuse.Attr) error {
+ glog.V(3).Infof("dir Attr %s", dir.Path)
+
// https://github.com/bazil/fuse/issues/196
attr.Valid = time.Second
@@ -38,36 +44,28 @@ func (dir *Dir) Attr(ctx context.Context, attr *fuse.Attr) error {
return nil
}
- item := dir.wfs.listDirectoryEntriesCache.Get(dir.Path)
- if item != nil && !item.Expired() {
- entry := item.Value().(*filer_pb.Entry)
-
- attr.Mtime = time.Unix(entry.Attributes.Mtime, 0)
- attr.Ctime = time.Unix(entry.Attributes.Crtime, 0)
- attr.Mode = os.FileMode(entry.Attributes.FileMode)
- attr.Gid = entry.Attributes.Gid
- attr.Uid = entry.Attributes.Uid
-
- return nil
- }
-
- entry, err := filer2.GetEntry(ctx, dir.wfs, dir.Path)
- if err != nil {
- glog.V(2).Infof("read dir %s attr: %v, error: %v", dir.Path, dir.attributes, err)
+ if err := dir.maybeLoadEntry(ctx); err != nil {
return err
}
- dir.attributes = entry.Attributes
- glog.V(2).Infof("dir %s: %v perm: %v", dir.Path, dir.attributes, os.FileMode(dir.attributes.FileMode))
+ attr.Mode = os.FileMode(dir.entry.Attributes.FileMode) | os.ModeDir
+ attr.Mtime = time.Unix(dir.entry.Attributes.Mtime, 0)
+ attr.Ctime = time.Unix(dir.entry.Attributes.Crtime, 0)
+ attr.Gid = dir.entry.Attributes.Gid
+ attr.Uid = dir.entry.Attributes.Uid
+
+ return nil
+}
- attr.Mode = os.FileMode(dir.attributes.FileMode) | os.ModeDir
+func (dir *Dir) Getxattr(ctx context.Context, req *fuse.GetxattrRequest, resp *fuse.GetxattrResponse) error {
- attr.Mtime = time.Unix(dir.attributes.Mtime, 0)
- attr.Ctime = time.Unix(dir.attributes.Crtime, 0)
- attr.Gid = dir.attributes.Gid
- attr.Uid = dir.attributes.Uid
+ glog.V(4).Infof("dir Getxattr %s", dir.Path)
- return nil
+ if err := dir.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ return getxattr(dir.entry, req, resp)
}
func (dir *Dir) setRootDirAttributes(attr *fuse.Attr) {
@@ -101,7 +99,7 @@ func (dir *Dir) Create(ctx context.Context, req *fuse.CreateRequest,
Attributes: &filer_pb.FuseAttributes{
Mtime: time.Now().Unix(),
Crtime: time.Now().Unix(),
- FileMode: uint32(req.Mode),
+ FileMode: uint32(req.Mode &^ dir.wfs.option.Umask),
Uid: req.Uid,
Gid: req.Gid,
Collection: dir.wfs.option.Collection,
@@ -146,7 +144,7 @@ func (dir *Dir) Mkdir(ctx context.Context, req *fuse.MkdirRequest) (fs.Node, err
Attributes: &filer_pb.FuseAttributes{
Mtime: time.Now().Unix(),
Crtime: time.Now().Unix(),
- FileMode: uint32(req.Mode),
+ FileMode: uint32(req.Mode &^ dir.wfs.option.Umask),
Uid: req.Uid,
Gid: req.Gid,
},
@@ -172,6 +170,8 @@ func (dir *Dir) Mkdir(ctx context.Context, req *fuse.MkdirRequest) (fs.Node, err
func (dir *Dir) Lookup(ctx context.Context, req *fuse.LookupRequest, resp *fuse.LookupResponse) (node fs.Node, err error) {
+ glog.V(4).Infof("dir Lookup %s: %s", dir.Path, req.Name)
+
var entry *filer_pb.Entry
fullFilePath := path.Join(dir.Path, req.Name)
@@ -181,15 +181,21 @@ func (dir *Dir) Lookup(ctx context.Context, req *fuse.LookupRequest, resp *fuse.
}
if entry == nil {
+ glog.V(3).Infof("dir Lookup cache miss %s", fullFilePath)
entry, err = filer2.GetEntry(ctx, dir.wfs, fullFilePath)
if err != nil {
return nil, err
}
+ if entry != nil {
+ dir.wfs.listDirectoryEntriesCache.Set(fullFilePath, entry, 5*time.Minute)
+ }
+ } else {
+ glog.V(4).Infof("dir Lookup cache hit %s", fullFilePath)
}
if entry != nil {
if entry.IsDirectory {
- node = &Dir{Path: path.Join(dir.Path, req.Name), wfs: dir.wfs, attributes: entry.Attributes}
+ node = &Dir{Path: path.Join(dir.Path, req.Name), wfs: dir.wfs, entry: entry}
} else {
node = dir.newFile(req.Name, entry)
}
@@ -209,52 +215,24 @@ func (dir *Dir) Lookup(ctx context.Context, req *fuse.LookupRequest, resp *fuse.
func (dir *Dir) ReadDirAll(ctx context.Context) (ret []fuse.Dirent, err error) {
- err = dir.wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
-
- paginationLimit := 1024
- remaining := dir.wfs.option.DirListingLimit
-
- lastEntryName := ""
-
- for remaining >= 0 {
-
- request := &filer_pb.ListEntriesRequest{
- Directory: dir.Path,
- StartFromFileName: lastEntryName,
- Limit: uint32(paginationLimit),
- }
-
- glog.V(4).Infof("read directory: %v", request)
- resp, err := client.ListEntries(ctx, request)
- if err != nil {
- glog.V(0).Infof("list %s: %v", dir.Path, err)
- return fuse.EIO
- }
-
- cacheTtl := estimatedCacheTtl(len(resp.Entries))
-
- for _, entry := range resp.Entries {
- if entry.IsDirectory {
- dirent := fuse.Dirent{Name: entry.Name, Type: fuse.DT_Dir}
- ret = append(ret, dirent)
- } else {
- dirent := fuse.Dirent{Name: entry.Name, Type: fuse.DT_File}
- ret = append(ret, dirent)
- }
- dir.wfs.listDirectoryEntriesCache.Set(path.Join(dir.Path, entry.Name), entry, cacheTtl)
- lastEntryName = entry.Name
- }
-
- remaining -= len(resp.Entries)
+ glog.V(3).Infof("dir ReadDirAll %s", dir.Path)
- if len(resp.Entries) < paginationLimit {
- break
- }
+ cacheTtl := 5 * time.Minute
+ readErr := filer2.ReadDirAllEntries(ctx, dir.wfs, dir.Path, "", func(entry *filer_pb.Entry, isLast bool) {
+ if entry.IsDirectory {
+ dirent := fuse.Dirent{Name: entry.Name, Type: fuse.DT_Dir}
+ ret = append(ret, dirent)
+ } else {
+ dirent := fuse.Dirent{Name: entry.Name, Type: fuse.DT_File}
+ ret = append(ret, dirent)
}
-
- return nil
+ dir.wfs.listDirectoryEntriesCache.Set(path.Join(dir.Path, entry.Name), entry, cacheTtl)
})
+ if readErr != nil {
+ glog.V(0).Infof("list %s: %v", dir.Path, err)
+ return ret, fuse.EIO
+ }
return ret, err
}
@@ -278,6 +256,8 @@ func (dir *Dir) removeOneFile(ctx context.Context, req *fuse.RemoveRequest) erro
dir.wfs.deleteFileChunks(ctx, entry.Chunks)
+ dir.wfs.listDirectoryEntriesCache.Delete(path.Join(dir.Path, req.Name))
+
return dir.wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.DeleteEntryRequest{
@@ -293,8 +273,6 @@ func (dir *Dir) removeOneFile(ctx context.Context, req *fuse.RemoveRequest) erro
return fuse.ENOENT
}
- dir.wfs.listDirectoryEntriesCache.Delete(path.Join(dir.Path, req.Name))
-
return nil
})
@@ -302,6 +280,8 @@ func (dir *Dir) removeOneFile(ctx context.Context, req *fuse.RemoveRequest) erro
func (dir *Dir) removeFolder(ctx context.Context, req *fuse.RemoveRequest) error {
+ dir.wfs.listDirectoryEntriesCache.Delete(path.Join(dir.Path, req.Name))
+
return dir.wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.DeleteEntryRequest{
@@ -317,8 +297,6 @@ func (dir *Dir) removeFolder(ctx context.Context, req *fuse.RemoveRequest) error
return fuse.ENOENT
}
- dir.wfs.listDirectoryEntriesCache.Delete(path.Join(dir.Path, req.Name))
-
return nil
})
@@ -326,66 +304,118 @@ func (dir *Dir) removeFolder(ctx context.Context, req *fuse.RemoveRequest) error
func (dir *Dir) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *fuse.SetattrResponse) error {
- if dir.attributes == nil {
- return nil
+ if err := dir.maybeLoadEntry(ctx); err != nil {
+ return err
}
glog.V(3).Infof("%v dir setattr %+v, fh=%d", dir.Path, req, req.Handle)
if req.Valid.Mode() {
- dir.attributes.FileMode = uint32(req.Mode)
+ dir.entry.Attributes.FileMode = uint32(req.Mode)
}
if req.Valid.Uid() {
- dir.attributes.Uid = req.Uid
+ dir.entry.Attributes.Uid = req.Uid
}
if req.Valid.Gid() {
- dir.attributes.Gid = req.Gid
+ dir.entry.Attributes.Gid = req.Gid
}
if req.Valid.Mtime() {
- dir.attributes.Mtime = req.Mtime.Unix()
+ dir.entry.Attributes.Mtime = req.Mtime.Unix()
+ }
+
+ dir.wfs.listDirectoryEntriesCache.Delete(dir.Path)
+
+ return dir.saveEntry(ctx)
+
+}
+
+func (dir *Dir) Setxattr(ctx context.Context, req *fuse.SetxattrRequest) error {
+
+ glog.V(4).Infof("dir Setxattr %s: %s", dir.Path, req.Name)
+
+ if err := dir.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ if err := setxattr(dir.entry, req); err != nil {
+ return err
+ }
+
+ dir.wfs.listDirectoryEntriesCache.Delete(dir.Path)
+
+ return dir.saveEntry(ctx)
+
+}
+
+func (dir *Dir) Removexattr(ctx context.Context, req *fuse.RemovexattrRequest) error {
+
+ glog.V(4).Infof("dir Removexattr %s: %s", dir.Path, req.Name)
+
+ if err := dir.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ if err := removexattr(dir.entry, req); err != nil {
+ return err
+ }
+
+ dir.wfs.listDirectoryEntriesCache.Delete(dir.Path)
+
+ return dir.saveEntry(ctx)
+
+}
+
+func (dir *Dir) Listxattr(ctx context.Context, req *fuse.ListxattrRequest, resp *fuse.ListxattrResponse) error {
+
+ glog.V(4).Infof("dir Listxattr %s", dir.Path)
+
+ if err := dir.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ if err := listxattr(dir.entry, req, resp); err != nil {
+ return err
}
+ return nil
+
+}
+
+func (dir *Dir) maybeLoadEntry(ctx context.Context) error {
+ if dir.entry == nil {
+ parentDirPath, name := filer2.FullPath(dir.Path).DirAndName()
+ entry, err := dir.wfs.maybeLoadEntry(ctx, parentDirPath, name)
+ if err != nil {
+ return err
+ }
+ if entry == nil {
+ return fuse.ENOENT
+ }
+ dir.entry = entry
+ }
+ return nil
+}
+
+func (dir *Dir) saveEntry(ctx context.Context) error {
+
parentDir, name := filer2.FullPath(dir.Path).DirAndName()
+
return dir.wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.UpdateEntryRequest{
Directory: parentDir,
- Entry: &filer_pb.Entry{
- Name: name,
- Attributes: dir.attributes,
- },
+ Entry: dir.entry,
}
- glog.V(1).Infof("set attr directory entry: %v", request)
+ glog.V(1).Infof("save dir entry: %v", request)
_, err := client.UpdateEntry(ctx, request)
if err != nil {
- glog.V(0).Infof("UpdateEntry %s: %v", dir.Path, err)
+ glog.V(0).Infof("UpdateEntry dir %s/%s: %v", parentDir, name, err)
return fuse.EIO
}
- dir.wfs.listDirectoryEntriesCache.Delete(dir.Path)
-
return nil
})
-
-}
-
-func estimatedCacheTtl(numEntries int) time.Duration {
- if numEntries < 100 {
- // 30 ms per entry
- return 3 * time.Second
- }
- if numEntries < 1000 {
- // 10 ms per entry
- return 10 * time.Second
- }
- if numEntries < 10000 {
- // 10 ms per entry
- return 100 * time.Second
- }
-
- // 2 ms per entry
- return time.Duration(numEntries*2) * time.Millisecond
}
diff --git a/weed/filesys/dir_link.go b/weed/filesys/dir_link.go
index 92cf04d58..8e60872d3 100644
--- a/weed/filesys/dir_link.go
+++ b/weed/filesys/dir_link.go
@@ -27,7 +27,7 @@ func (dir *Dir) Symlink(ctx context.Context, req *fuse.SymlinkRequest) (fs.Node,
Attributes: &filer_pb.FuseAttributes{
Mtime: time.Now().Unix(),
Crtime: time.Now().Unix(),
- FileMode: uint32(os.FileMode(0755) | os.ModeSymlink),
+ FileMode: uint32((os.FileMode(0777) | os.ModeSymlink) &^ dir.wfs.option.Umask),
Uid: req.Uid,
Gid: req.Gid,
SymlinkTarget: req.Target,
@@ -51,7 +51,7 @@ func (dir *Dir) Symlink(ctx context.Context, req *fuse.SymlinkRequest) (fs.Node,
func (file *File) Readlink(ctx context.Context, req *fuse.ReadlinkRequest) (string, error) {
- if err := file.maybeLoadAttributes(ctx); err != nil {
+ if err := file.maybeLoadEntry(ctx); err != nil {
return "", err
}
diff --git a/weed/filesys/dirty_page.go b/weed/filesys/dirty_page.go
index baee412b2..35d8f249a 100644
--- a/weed/filesys/dirty_page.go
+++ b/weed/filesys/dirty_page.go
@@ -192,7 +192,7 @@ func (pages *ContinuousDirtyPages) saveToStorage(ctx context.Context, buf []byte
fileUrl := fmt.Sprintf("http://%s/%s", host, fileId)
bufReader := bytes.NewReader(buf)
- uploadResult, err := operation.Upload(fileUrl, pages.f.Name, bufReader, false, "application/octet-stream", nil, auth)
+ uploadResult, err := operation.Upload(fileUrl, pages.f.Name, bufReader, false, "", nil, auth)
if err != nil {
glog.V(0).Infof("upload data %v to %s: %v", pages.f.Name, fileUrl, err)
return nil, fmt.Errorf("upload data: %v", err)
diff --git a/weed/filesys/file.go b/weed/filesys/file.go
index 1b359ebbe..afe78ee0f 100644
--- a/weed/filesys/file.go
+++ b/weed/filesys/file.go
@@ -20,6 +20,10 @@ var _ = fs.Node(&File{})
var _ = fs.NodeOpener(&File{})
var _ = fs.NodeFsyncer(&File{})
var _ = fs.NodeSetattrer(&File{})
+var _ = fs.NodeGetxattrer(&File{})
+var _ = fs.NodeSetxattrer(&File{})
+var _ = fs.NodeRemovexattrer(&File{})
+var _ = fs.NodeListxattrer(&File{})
type File struct {
Name string
@@ -36,7 +40,9 @@ func (file *File) fullpath() string {
func (file *File) Attr(ctx context.Context, attr *fuse.Attr) error {
- if err := file.maybeLoadAttributes(ctx); err != nil {
+ glog.V(4).Infof("file Attr %s", file.fullpath())
+
+ if err := file.maybeLoadEntry(ctx); err != nil {
return err
}
@@ -52,9 +58,20 @@ func (file *File) Attr(ctx context.Context, attr *fuse.Attr) error {
}
+func (file *File) Getxattr(ctx context.Context, req *fuse.GetxattrRequest, resp *fuse.GetxattrResponse) error {
+
+ glog.V(4).Infof("file Getxattr %s", file.fullpath())
+
+ if err := file.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ return getxattr(file.entry, req, resp)
+}
+
func (file *File) Open(ctx context.Context, req *fuse.OpenRequest, resp *fuse.OpenResponse) (fs.Handle, error) {
- glog.V(3).Infof("%v file open %+v", file.fullpath(), req)
+ glog.V(4).Infof("file %v open %+v", file.fullpath(), req)
file.isOpen = true
@@ -70,7 +87,7 @@ func (file *File) Open(ctx context.Context, req *fuse.OpenRequest, resp *fuse.Op
func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *fuse.SetattrResponse) error {
- if err := file.maybeLoadAttributes(ctx); err != nil {
+ if err := file.maybeLoadEntry(ctx); err != nil {
return err
}
@@ -109,66 +126,80 @@ func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *f
return nil
}
- return file.wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+ file.wfs.listDirectoryEntriesCache.Delete(file.fullpath())
- request := &filer_pb.UpdateEntryRequest{
- Directory: file.dir.Path,
- Entry: file.entry,
- }
+ return file.saveEntry(ctx)
- glog.V(1).Infof("set attr file entry: %v", request)
- _, err := client.UpdateEntry(ctx, request)
- if err != nil {
- glog.V(0).Infof("UpdateEntry file %s/%s: %v", file.dir.Path, file.Name, err)
- return fuse.EIO
- }
+}
- return nil
- })
+func (file *File) Setxattr(ctx context.Context, req *fuse.SetxattrRequest) error {
+
+ glog.V(4).Infof("file Setxattr %s: %s", file.fullpath(), req.Name)
+
+ if err := file.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ if err := setxattr(file.entry, req); err != nil {
+ return err
+ }
+
+ file.wfs.listDirectoryEntriesCache.Delete(file.fullpath())
+
+ return file.saveEntry(ctx)
}
-func (file *File) Fsync(ctx context.Context, req *fuse.FsyncRequest) error {
- // fsync works at OS level
- // write the file chunks to the filerGrpcAddress
- glog.V(3).Infof("%s/%s fsync file %+v", file.dir.Path, file.Name, req)
+func (file *File) Removexattr(ctx context.Context, req *fuse.RemovexattrRequest) error {
+
+ glog.V(4).Infof("file Removexattr %s: %s", file.fullpath(), req.Name)
+
+ if err := file.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ if err := removexattr(file.entry, req); err != nil {
+ return err
+ }
+
+ file.wfs.listDirectoryEntriesCache.Delete(file.fullpath())
+
+ return file.saveEntry(ctx)
- return nil
}
-func (file *File) maybeLoadAttributes(ctx context.Context) error {
- if file.entry == nil || !file.isOpen {
- item := file.wfs.listDirectoryEntriesCache.Get(file.fullpath())
- if item != nil && !item.Expired() {
- entry := item.Value().(*filer_pb.Entry)
- file.setEntry(entry)
- // glog.V(1).Infof("file attr read cached %v attributes", file.Name)
- } else {
- err := file.wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+func (file *File) Listxattr(ctx context.Context, req *fuse.ListxattrRequest, resp *fuse.ListxattrResponse) error {
- request := &filer_pb.LookupDirectoryEntryRequest{
- Name: file.Name,
- Directory: file.dir.Path,
- }
+ glog.V(4).Infof("file Listxattr %s", file.fullpath())
- resp, err := client.LookupDirectoryEntry(ctx, request)
- if err != nil {
- glog.V(3).Infof("file attr read file %v: %v", request, err)
- return fuse.ENOENT
- }
+ if err := file.maybeLoadEntry(ctx); err != nil {
+ return err
+ }
+
+ if err := listxattr(file.entry, req, resp); err != nil {
+ return err
+ }
- file.setEntry(resp.Entry)
+ return nil
- glog.V(3).Infof("file attr %v %+v: %d", file.fullpath(), file.entry.Attributes, filer2.TotalSize(file.entry.Chunks))
+}
- // file.wfs.listDirectoryEntriesCache.Set(file.fullpath(), file.entry, file.wfs.option.EntryCacheTtl)
+func (file *File) Fsync(ctx context.Context, req *fuse.FsyncRequest) error {
+ // fsync works at OS level
+ // write the file chunks to the filerGrpcAddress
+ glog.V(3).Infof("%s/%s fsync file %+v", file.dir.Path, file.Name, req)
- return nil
- })
+ return nil
+}
- if err != nil {
- return err
- }
+func (file *File) maybeLoadEntry(ctx context.Context) error {
+ if file.entry == nil || !file.isOpen {
+ entry, err := file.wfs.maybeLoadEntry(ctx, file.dir.Path, file.Name)
+ if err != nil {
+ return err
+ }
+ if entry != nil {
+ file.setEntry(entry)
}
}
return nil
@@ -203,3 +234,22 @@ func (file *File) setEntry(entry *filer_pb.Entry) {
file.entry = entry
file.entryViewCache = filer2.NonOverlappingVisibleIntervals(file.entry.Chunks)
}
+
+func (file *File) saveEntry(ctx context.Context) error {
+ return file.wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+
+ request := &filer_pb.UpdateEntryRequest{
+ Directory: file.dir.Path,
+ Entry: file.entry,
+ }
+
+ glog.V(1).Infof("save file entry: %v", request)
+ _, err := client.UpdateEntry(ctx, request)
+ if err != nil {
+ glog.V(0).Infof("UpdateEntry file %s/%s: %v", file.dir.Path, file.Name, err)
+ return fuse.EIO
+ }
+
+ return nil
+ })
+}
diff --git a/weed/filesys/filehandle.go b/weed/filesys/filehandle.go
index ceec50e13..101f5c056 100644
--- a/weed/filesys/filehandle.go
+++ b/weed/filesys/filehandle.go
@@ -92,18 +92,19 @@ func (fh *FileHandle) Write(ctx context.Context, req *fuse.WriteRequest, resp *f
if req.Offset == 0 {
// detect mime type
- var possibleExt string
- fh.contentType, possibleExt = mimetype.Detect(req.Data)
- if ext := path.Ext(fh.f.Name); ext != possibleExt {
+ detectedMIME := mimetype.Detect(req.Data)
+ fh.contentType = detectedMIME.String()
+ if ext := path.Ext(fh.f.Name); ext != detectedMIME.Extension() {
fh.contentType = mime.TypeByExtension(ext)
}
fh.dirtyMetadata = true
}
- fh.f.addChunks(chunks)
-
if len(chunks) > 0 {
+
+ fh.f.addChunks(chunks)
+
fh.dirtyMetadata = true
}
@@ -148,7 +149,7 @@ func (fh *FileHandle) Flush(ctx context.Context, req *fuse.FlushRequest) error {
fh.f.entry.Attributes.Gid = req.Gid
fh.f.entry.Attributes.Mtime = time.Now().Unix()
fh.f.entry.Attributes.Crtime = time.Now().Unix()
- fh.f.entry.Attributes.FileMode = uint32(0770)
+ fh.f.entry.Attributes.FileMode = uint32(0777 &^ fh.f.wfs.option.Umask)
}
request := &filer_pb.CreateEntryRequest{
diff --git a/weed/filesys/wfs.go b/weed/filesys/wfs.go
index 9018c36ed..e924783ec 100644
--- a/weed/filesys/wfs.go
+++ b/weed/filesys/wfs.go
@@ -8,13 +8,14 @@ import (
"sync"
"time"
+ "github.com/karlseguin/ccache"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/karlseguin/ccache"
"github.com/seaweedfs/fuse"
"github.com/seaweedfs/fuse/fs"
- "google.golang.org/grpc"
)
type Option struct {
@@ -26,8 +27,9 @@ type Option struct {
TtlSec int32
ChunkSizeLimit int64
DataCenter string
- DirListingLimit int
+ DirListCacheLimit int64
EntryCacheTtl time.Duration
+ Umask os.FileMode
MountUid uint32
MountGid uint32
@@ -59,7 +61,7 @@ type statsCache struct {
func NewSeaweedFileSystem(option *Option) *WFS {
wfs := &WFS{
option: option,
- listDirectoryEntriesCache: ccache.New(ccache.Configure().MaxSize(1024 * 8).ItemsToPrune(100)),
+ listDirectoryEntriesCache: ccache.New(ccache.Configure().MaxSize(option.DirListCacheLimit * 3).ItemsToPrune(100)),
pathToHandleIndex: make(map[string]int),
bufPool: sync.Pool{
New: func() interface{} {
diff --git a/weed/filesys/xattr.go b/weed/filesys/xattr.go
new file mode 100644
index 000000000..3c0ba164a
--- /dev/null
+++ b/weed/filesys/xattr.go
@@ -0,0 +1,142 @@
+package filesys
+
+import (
+ "context"
+ "path/filepath"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+ "github.com/seaweedfs/fuse"
+)
+
+func getxattr(entry *filer_pb.Entry, req *fuse.GetxattrRequest, resp *fuse.GetxattrResponse) error {
+
+ if entry == nil {
+ return fuse.ErrNoXattr
+ }
+ if entry.Extended == nil {
+ return fuse.ErrNoXattr
+ }
+ data, found := entry.Extended[req.Name]
+ if !found {
+ return fuse.ErrNoXattr
+ }
+ if req.Position < uint32(len(data)) {
+ size := req.Size
+ if req.Position+size >= uint32(len(data)) {
+ size = uint32(len(data)) - req.Position
+ }
+ if size == 0 {
+ resp.Xattr = data[req.Position:]
+ } else {
+ resp.Xattr = data[req.Position : req.Position+size]
+ }
+ }
+
+ return nil
+
+}
+
+func setxattr(entry *filer_pb.Entry, req *fuse.SetxattrRequest) error {
+
+ if entry == nil {
+ return fuse.EIO
+ }
+
+ if entry.Extended == nil {
+ entry.Extended = make(map[string][]byte)
+ }
+ data, _ := entry.Extended[req.Name]
+
+ newData := make([]byte, int(req.Position)+len(req.Xattr))
+
+ copy(newData, data)
+
+ copy(newData[int(req.Position):], req.Xattr)
+
+ entry.Extended[req.Name] = newData
+
+ return nil
+
+}
+
+func removexattr(entry *filer_pb.Entry, req *fuse.RemovexattrRequest) error {
+
+ if entry == nil {
+ return fuse.ErrNoXattr
+ }
+
+ if entry.Extended == nil {
+ return fuse.ErrNoXattr
+ }
+
+ _, found := entry.Extended[req.Name]
+
+ if !found {
+ return fuse.ErrNoXattr
+ }
+
+ delete(entry.Extended, req.Name)
+
+ return nil
+
+}
+
+func listxattr(entry *filer_pb.Entry, req *fuse.ListxattrRequest, resp *fuse.ListxattrResponse) error {
+
+ if entry == nil {
+ return fuse.EIO
+ }
+
+ for k := range entry.Extended {
+ resp.Append(k)
+ }
+
+ size := req.Size
+ if req.Position+size >= uint32(len(resp.Xattr)) {
+ size = uint32(len(resp.Xattr)) - req.Position
+ }
+
+ if size == 0 {
+ resp.Xattr = resp.Xattr[req.Position:]
+ } else {
+ resp.Xattr = resp.Xattr[req.Position : req.Position+size]
+ }
+
+ return nil
+
+}
+
+func (wfs *WFS) maybeLoadEntry(ctx context.Context, dir, name string) (entry *filer_pb.Entry, err error) {
+
+ fullpath := filepath.Join(dir, name)
+ item := wfs.listDirectoryEntriesCache.Get(fullpath)
+ if item != nil && !item.Expired() {
+ entry = item.Value().(*filer_pb.Entry)
+ return
+ }
+ glog.V(3).Infof("read entry cache miss %s", fullpath)
+
+ err = wfs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+
+ request := &filer_pb.LookupDirectoryEntryRequest{
+ Name: name,
+ Directory: dir,
+ }
+
+ resp, err := client.LookupDirectoryEntry(ctx, request)
+ if err != nil {
+ glog.V(3).Infof("file attr read file %v: %v", request, err)
+ return fuse.ENOENT
+ }
+
+ entry = resp.Entry
+ if entry != nil {
+ wfs.listDirectoryEntriesCache.Set(fullpath, entry, wfs.option.EntryCacheTtl)
+ }
+
+ return nil
+ })
+
+ return
+}
diff --git a/weed/glide.yaml b/weed/glide.yaml
deleted file mode 100644
index ef64b3a3c..000000000
--- a/weed/glide.yaml
+++ /dev/null
@@ -1,117 +0,0 @@
-package: github.com/chrislusf/seaweedfs/weed
-import:
-- package: cloud.google.com/go
- version: ^0.40.0
- subpackages:
- - pubsub
- - storage
-- package: github.com/Azure/azure-storage-blob-go
- version: ^0.7.0
- subpackages:
- - azblob
-- package: github.com/Shopify/sarama
- version: ^1.22.1
-- package: github.com/aws/aws-sdk-go
- version: ^1.20.12
- subpackages:
- - aws
- - aws/awserr
- - aws/credentials
- - aws/session
- - service/s3
- - service/s3/s3iface
- - service/sqs
-- package: github.com/chrislusf/raft
- subpackages:
- - protobuf
-- package: github.com/dgrijalva/jwt-go
- version: ^3.2.0
-- package: github.com/disintegration/imaging
- version: ^1.6.0
-- package: github.com/dustin/go-humanize
- version: ^1.0.0
-- package: github.com/gabriel-vasile/mimetype
- version: ^0.3.14
-- package: github.com/go-redis/redis
- version: ^6.15.3
-- package: github.com/go-sql-driver/mysql
- version: ^1.4.1
-- package: github.com/gocql/gocql
-- package: github.com/golang/protobuf
- version: ^1.3.1
- subpackages:
- - proto
-- package: github.com/google/btree
- version: ^1.0.0
-- package: github.com/gorilla/mux
- version: ^1.7.3
-- package: github.com/jacobsa/daemonize
-- package: github.com/kardianos/osext
-- package: github.com/karlseguin/ccache
- version: ^2.0.3
-- package: github.com/klauspost/crc32
- version: ^1.2.0
-- package: github.com/klauspost/reedsolomon
- version: ^1.9.2
-- package: github.com/kurin/blazer
- version: ^0.5.3
- subpackages:
- - b2
-- package: github.com/lib/pq
- version: ^1.1.1
-- package: github.com/peterh/liner
- version: ^1.1.0
-- package: github.com/prometheus/client_golang
- version: ^1.0.0
- subpackages:
- - prometheus
- - prometheus/push
-- package: github.com/rakyll/statik
- version: ^0.1.6
- subpackages:
- - fs
-- package: github.com/rwcarlsen/goexif
- subpackages:
- - exif
-- package: github.com/satori/go.uuid
- version: ^1.2.0
-- package: github.com/seaweedfs/fuse
- subpackages:
- - fs
-- package: github.com/spf13/viper
- version: ^1.4.0
-- package: github.com/syndtr/goleveldb
- version: ^1.0.0
- subpackages:
- - leveldb
- - leveldb/opt
- - leveldb/util
-- package: github.com/willf/bloom
- version: ^2.0.3
-- package: gocloud.dev
- version: ^0.15.0
- subpackages:
- - pubsub
- - pubsub/awssnssqs
- - pubsub/azuresb
- - pubsub/gcppubsub
- - pubsub/natspubsub
- - pubsub/rabbitpubsub
-- package: golang.org/x/net
- subpackages:
- - context
- - webdav
-- package: golang.org/x/tools
- subpackages:
- - godoc/util
-- package: google.golang.org/api
- version: ^0.7.0
- subpackages:
- - option
-- package: google.golang.org/grpc
- version: ^1.21.1
- subpackages:
- - credentials
- - keepalive
- - peer
- - reflection
diff --git a/weed/notification/aws_sqs/aws_sqs_pub.go b/weed/notification/aws_sqs/aws_sqs_pub.go
index c1af7f27a..4c1302abb 100644
--- a/weed/notification/aws_sqs/aws_sqs_pub.go
+++ b/weed/notification/aws_sqs/aws_sqs_pub.go
@@ -38,13 +38,13 @@ func (k *AwsSqsPub) Initialize(configuration util.Configuration) (err error) {
)
}
-func (k *AwsSqsPub) initialize(awsAccessKeyId, aswSecretAccessKey, region, queueName string) (err error) {
+func (k *AwsSqsPub) initialize(awsAccessKeyId, awsSecretAccessKey, region, queueName string) (err error) {
config := &aws.Config{
Region: aws.String(region),
}
- if awsAccessKeyId != "" && aswSecretAccessKey != "" {
- config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, aswSecretAccessKey, "")
+ if awsAccessKeyId != "" && awsSecretAccessKey != "" {
+ config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, awsSecretAccessKey, "")
}
sess, err := session.NewSession(config)
diff --git a/weed/notification/kafka/kafka_queue.go b/weed/notification/kafka/kafka_queue.go
index 830709a51..fd545722b 100644
--- a/weed/notification/kafka/kafka_queue.go
+++ b/weed/notification/kafka/kafka_queue.go
@@ -76,7 +76,7 @@ func (k *KafkaQueue) handleError() {
for {
err := <-k.producer.Errors()
if err != nil {
- glog.Errorf("producer message error, partition:%d offset:%d key:%v valus:%s error(%v) topic:%s", err.Msg.Partition, err.Msg.Offset, err.Msg.Key, err.Msg.Value, err.Err, k.topic)
+ glog.Errorf("producer message error, partition:%d offset:%d key:%v value:%s error(%v) topic:%s", err.Msg.Partition, err.Msg.Offset, err.Msg.Key, err.Msg.Value, err.Err, k.topic)
}
}
}
diff --git a/weed/operation/assign_file_id.go b/weed/operation/assign_file_id.go
index 4c50eaa26..2dfa44483 100644
--- a/weed/operation/assign_file_id.go
+++ b/weed/operation/assign_file_id.go
@@ -11,13 +11,14 @@ import (
)
type VolumeAssignRequest struct {
- Count uint64
- Replication string
- Collection string
- Ttl string
- DataCenter string
- Rack string
- DataNode string
+ Count uint64
+ Replication string
+ Collection string
+ Ttl string
+ DataCenter string
+ Rack string
+ DataNode string
+ WritableVolumeCount uint32
}
type AssignResult struct {
@@ -46,13 +47,14 @@ func Assign(server string, grpcDialOption grpc.DialOption, primaryRequest *Volum
lastError = WithMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
req := &master_pb.AssignRequest{
- Count: primaryRequest.Count,
- Replication: primaryRequest.Replication,
- Collection: primaryRequest.Collection,
- Ttl: primaryRequest.Ttl,
- DataCenter: primaryRequest.DataCenter,
- Rack: primaryRequest.Rack,
- DataNode: primaryRequest.DataNode,
+ Count: primaryRequest.Count,
+ Replication: primaryRequest.Replication,
+ Collection: primaryRequest.Collection,
+ Ttl: primaryRequest.Ttl,
+ DataCenter: primaryRequest.DataCenter,
+ Rack: primaryRequest.Rack,
+ DataNode: primaryRequest.DataNode,
+ WritableVolumeCount: primaryRequest.WritableVolumeCount,
}
resp, grpcErr := masterClient.Assign(context.Background(), req)
if grpcErr != nil {
diff --git a/weed/operation/delete_content.go b/weed/operation/delete_content.go
index 6d84be76f..358399324 100644
--- a/weed/operation/delete_content.go
+++ b/weed/operation/delete_content.go
@@ -107,7 +107,7 @@ func DeleteFilesWithLookupVolumeId(grpcDialOption grpc.DialOption, fileIds []str
ret = append(ret, result...)
}
- glog.V(0).Infof("deleted %d items", len(ret))
+ glog.V(1).Infof("deleted %d items", len(ret))
return ret, err
}
diff --git a/weed/operation/submit.go b/weed/operation/submit.go
index bdf59d966..62f067430 100644
--- a/weed/operation/submit.go
+++ b/weed/operation/submit.go
@@ -203,7 +203,7 @@ func upload_one_chunk(filename string, reader io.Reader, master,
) (size uint32, e error) {
glog.V(4).Info("Uploading part ", filename, " to ", fileUrl, "...")
uploadResult, uploadError := Upload(fileUrl, filename, reader, false,
- "application/octet-stream", nil, jwt)
+ "", nil, jwt)
if uploadError != nil {
return 0, uploadError
}
diff --git a/weed/pb/filer.proto b/weed/pb/filer.proto
index d72bced12..ef847cbe7 100644
--- a/weed/pb/filer.proto
+++ b/weed/pb/filer.proto
@@ -12,7 +12,7 @@ service SeaweedFiler {
rpc LookupDirectoryEntry (LookupDirectoryEntryRequest) returns (LookupDirectoryEntryResponse) {
}
- rpc ListEntries (ListEntriesRequest) returns (ListEntriesResponse) {
+ rpc ListEntries (ListEntriesRequest) returns (stream ListEntriesResponse) {
}
rpc CreateEntry (CreateEntryRequest) returns (CreateEntryResponse) {
@@ -64,7 +64,7 @@ message ListEntriesRequest {
}
message ListEntriesResponse {
- repeated Entry entries = 1;
+ Entry entry = 1;
}
message Entry {
@@ -141,6 +141,7 @@ message DeleteEntryRequest {
// bool is_directory = 3;
bool is_delete_data = 4;
bool is_recursive = 5;
+ bool ignore_recursive_error = 6;
}
message DeleteEntryResponse {
diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go
index 1a35ad7c0..c8214aa94 100644
--- a/weed/pb/filer_pb/filer.pb.go
+++ b/weed/pb/filer_pb/filer.pb.go
@@ -151,7 +151,7 @@ func (m *ListEntriesRequest) GetLimit() uint32 {
}
type ListEntriesResponse struct {
- Entries []*Entry `protobuf:"bytes,1,rep,name=entries" json:"entries,omitempty"`
+ Entry *Entry `protobuf:"bytes,1,opt,name=entry" json:"entry,omitempty"`
}
func (m *ListEntriesResponse) Reset() { *m = ListEntriesResponse{} }
@@ -159,9 +159,9 @@ func (m *ListEntriesResponse) String() string { return proto.CompactT
func (*ListEntriesResponse) ProtoMessage() {}
func (*ListEntriesResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
-func (m *ListEntriesResponse) GetEntries() []*Entry {
+func (m *ListEntriesResponse) GetEntry() *Entry {
if m != nil {
- return m.Entries
+ return m.Entry
}
return nil
}
@@ -562,8 +562,9 @@ type DeleteEntryRequest struct {
Directory string `protobuf:"bytes,1,opt,name=directory" json:"directory,omitempty"`
Name string `protobuf:"bytes,2,opt,name=name" json:"name,omitempty"`
// bool is_directory = 3;
- IsDeleteData bool `protobuf:"varint,4,opt,name=is_delete_data,json=isDeleteData" json:"is_delete_data,omitempty"`
- IsRecursive bool `protobuf:"varint,5,opt,name=is_recursive,json=isRecursive" json:"is_recursive,omitempty"`
+ IsDeleteData bool `protobuf:"varint,4,opt,name=is_delete_data,json=isDeleteData" json:"is_delete_data,omitempty"`
+ IsRecursive bool `protobuf:"varint,5,opt,name=is_recursive,json=isRecursive" json:"is_recursive,omitempty"`
+ IgnoreRecursiveError bool `protobuf:"varint,6,opt,name=ignore_recursive_error,json=ignoreRecursiveError" json:"ignore_recursive_error,omitempty"`
}
func (m *DeleteEntryRequest) Reset() { *m = DeleteEntryRequest{} }
@@ -599,6 +600,13 @@ func (m *DeleteEntryRequest) GetIsRecursive() bool {
return false
}
+func (m *DeleteEntryRequest) GetIgnoreRecursiveError() bool {
+ if m != nil {
+ return m.IgnoreRecursiveError
+ }
+ return false
+}
+
type DeleteEntryResponse struct {
}
@@ -1028,7 +1036,7 @@ const _ = grpc.SupportPackageIsVersion4
type SeaweedFilerClient interface {
LookupDirectoryEntry(ctx context.Context, in *LookupDirectoryEntryRequest, opts ...grpc.CallOption) (*LookupDirectoryEntryResponse, error)
- ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (*ListEntriesResponse, error)
+ ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (SeaweedFiler_ListEntriesClient, error)
CreateEntry(ctx context.Context, in *CreateEntryRequest, opts ...grpc.CallOption) (*CreateEntryResponse, error)
UpdateEntry(ctx context.Context, in *UpdateEntryRequest, opts ...grpc.CallOption) (*UpdateEntryResponse, error)
DeleteEntry(ctx context.Context, in *DeleteEntryRequest, opts ...grpc.CallOption) (*DeleteEntryResponse, error)
@@ -1057,13 +1065,36 @@ func (c *seaweedFilerClient) LookupDirectoryEntry(ctx context.Context, in *Looku
return out, nil
}
-func (c *seaweedFilerClient) ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (*ListEntriesResponse, error) {
- out := new(ListEntriesResponse)
- err := grpc.Invoke(ctx, "/filer_pb.SeaweedFiler/ListEntries", in, out, c.cc, opts...)
+func (c *seaweedFilerClient) ListEntries(ctx context.Context, in *ListEntriesRequest, opts ...grpc.CallOption) (SeaweedFiler_ListEntriesClient, error) {
+ stream, err := grpc.NewClientStream(ctx, &_SeaweedFiler_serviceDesc.Streams[0], c.cc, "/filer_pb.SeaweedFiler/ListEntries", opts...)
if err != nil {
return nil, err
}
- return out, nil
+ x := &seaweedFilerListEntriesClient{stream}
+ if err := x.ClientStream.SendMsg(in); err != nil {
+ return nil, err
+ }
+ if err := x.ClientStream.CloseSend(); err != nil {
+ return nil, err
+ }
+ return x, nil
+}
+
+type SeaweedFiler_ListEntriesClient interface {
+ Recv() (*ListEntriesResponse, error)
+ grpc.ClientStream
+}
+
+type seaweedFilerListEntriesClient struct {
+ grpc.ClientStream
+}
+
+func (x *seaweedFilerListEntriesClient) Recv() (*ListEntriesResponse, error) {
+ m := new(ListEntriesResponse)
+ if err := x.ClientStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
}
func (c *seaweedFilerClient) CreateEntry(ctx context.Context, in *CreateEntryRequest, opts ...grpc.CallOption) (*CreateEntryResponse, error) {
@@ -1151,7 +1182,7 @@ func (c *seaweedFilerClient) GetFilerConfiguration(ctx context.Context, in *GetF
type SeaweedFilerServer interface {
LookupDirectoryEntry(context.Context, *LookupDirectoryEntryRequest) (*LookupDirectoryEntryResponse, error)
- ListEntries(context.Context, *ListEntriesRequest) (*ListEntriesResponse, error)
+ ListEntries(*ListEntriesRequest, SeaweedFiler_ListEntriesServer) error
CreateEntry(context.Context, *CreateEntryRequest) (*CreateEntryResponse, error)
UpdateEntry(context.Context, *UpdateEntryRequest) (*UpdateEntryResponse, error)
DeleteEntry(context.Context, *DeleteEntryRequest) (*DeleteEntryResponse, error)
@@ -1185,22 +1216,25 @@ func _SeaweedFiler_LookupDirectoryEntry_Handler(srv interface{}, ctx context.Con
return interceptor(ctx, in, info, handler)
}
-func _SeaweedFiler_ListEntries_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
- in := new(ListEntriesRequest)
- if err := dec(in); err != nil {
- return nil, err
- }
- if interceptor == nil {
- return srv.(SeaweedFilerServer).ListEntries(ctx, in)
- }
- info := &grpc.UnaryServerInfo{
- Server: srv,
- FullMethod: "/filer_pb.SeaweedFiler/ListEntries",
- }
- handler := func(ctx context.Context, req interface{}) (interface{}, error) {
- return srv.(SeaweedFilerServer).ListEntries(ctx, req.(*ListEntriesRequest))
+func _SeaweedFiler_ListEntries_Handler(srv interface{}, stream grpc.ServerStream) error {
+ m := new(ListEntriesRequest)
+ if err := stream.RecvMsg(m); err != nil {
+ return err
}
- return interceptor(ctx, in, info, handler)
+ return srv.(SeaweedFilerServer).ListEntries(m, &seaweedFilerListEntriesServer{stream})
+}
+
+type SeaweedFiler_ListEntriesServer interface {
+ Send(*ListEntriesResponse) error
+ grpc.ServerStream
+}
+
+type seaweedFilerListEntriesServer struct {
+ grpc.ServerStream
+}
+
+func (x *seaweedFilerListEntriesServer) Send(m *ListEntriesResponse) error {
+ return x.ServerStream.SendMsg(m)
}
func _SeaweedFiler_CreateEntry_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
@@ -1373,10 +1407,6 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
MethodName: "LookupDirectoryEntry",
Handler: _SeaweedFiler_LookupDirectoryEntry_Handler,
},
- {
- MethodName: "ListEntries",
- Handler: _SeaweedFiler_ListEntries_Handler,
- },
{
MethodName: "CreateEntry",
Handler: _SeaweedFiler_CreateEntry_Handler,
@@ -1414,111 +1444,119 @@ var _SeaweedFiler_serviceDesc = grpc.ServiceDesc{
Handler: _SeaweedFiler_GetFilerConfiguration_Handler,
},
},
- Streams: []grpc.StreamDesc{},
+ Streams: []grpc.StreamDesc{
+ {
+ StreamName: "ListEntries",
+ Handler: _SeaweedFiler_ListEntries_Handler,
+ ServerStreams: true,
+ },
+ },
Metadata: "filer.proto",
}
func init() { proto.RegisterFile("filer.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{
- // 1583 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0xdb, 0x6f, 0xdc, 0x44,
- 0x17, 0xaf, 0xf7, 0xee, 0xb3, 0xbb, 0x6d, 0x32, 0x49, 0xbf, 0xba, 0x9b, 0xcb, 0x97, 0x3a, 0x5f,
- 0xfb, 0xa5, 0xa2, 0x0a, 0x55, 0xe1, 0xa1, 0xa5, 0x42, 0xa2, 0xcd, 0x05, 0x45, 0xa4, 0x17, 0x39,
- 0x2d, 0x02, 0x21, 0x61, 0x39, 0xf6, 0xec, 0x66, 0x88, 0xed, 0x59, 0xec, 0x71, 0x92, 0xf2, 0x27,
- 0xf0, 0x82, 0xc4, 0x23, 0x12, 0xcf, 0xfc, 0x13, 0x88, 0x17, 0xc4, 0xbf, 0xc3, 0x23, 0xcf, 0x68,
- 0x2e, 0xf6, 0x8e, 0xd7, 0x9b, 0xa4, 0x08, 0xf5, 0xcd, 0x73, 0xae, 0xbf, 0x73, 0xe6, 0x5c, 0x66,
- 0x17, 0xba, 0x43, 0x12, 0xe2, 0x64, 0x73, 0x9c, 0x50, 0x46, 0x51, 0x47, 0x1c, 0xdc, 0xf1, 0xa1,
- 0xfd, 0x02, 0x96, 0xf6, 0x29, 0x3d, 0xce, 0xc6, 0xdb, 0x24, 0xc1, 0x3e, 0xa3, 0xc9, 0x9b, 0x9d,
- 0x98, 0x25, 0x6f, 0x1c, 0xfc, 0x6d, 0x86, 0x53, 0x86, 0x96, 0xc1, 0x0c, 0x72, 0x86, 0x65, 0xac,
- 0x19, 0x1b, 0xa6, 0x33, 0x21, 0x20, 0x04, 0x8d, 0xd8, 0x8b, 0xb0, 0x55, 0x13, 0x0c, 0xf1, 0x6d,
- 0xef, 0xc0, 0xf2, 0x6c, 0x83, 0xe9, 0x98, 0xc6, 0x29, 0x46, 0xb7, 0xa1, 0x89, 0x39, 0x41, 0x58,
- 0xeb, 0x3e, 0xb8, 0xb6, 0x99, 0x43, 0xd9, 0x94, 0x72, 0x92, 0x6b, 0xff, 0x66, 0x00, 0xda, 0x27,
- 0x29, 0xe3, 0x44, 0x82, 0xd3, 0xb7, 0xc3, 0xf3, 0x1f, 0x68, 0x8d, 0x13, 0x3c, 0x24, 0x67, 0x0a,
- 0x91, 0x3a, 0xa1, 0x7b, 0x30, 0x9f, 0x32, 0x2f, 0x61, 0xbb, 0x09, 0x8d, 0x76, 0x49, 0x88, 0x9f,
- 0x73, 0xd0, 0x75, 0x21, 0x52, 0x65, 0xa0, 0x4d, 0x40, 0x24, 0xf6, 0xc3, 0x2c, 0x25, 0x27, 0xf8,
- 0x20, 0xe7, 0x5a, 0x8d, 0x35, 0x63, 0xa3, 0xe3, 0xcc, 0xe0, 0xa0, 0x45, 0x68, 0x86, 0x24, 0x22,
- 0xcc, 0x6a, 0xae, 0x19, 0x1b, 0x7d, 0x47, 0x1e, 0xec, 0x4f, 0x60, 0xa1, 0x84, 0x5f, 0x85, 0x7f,
- 0x17, 0xda, 0x58, 0x92, 0x2c, 0x63, 0xad, 0x3e, 0x2b, 0x01, 0x39, 0xdf, 0xfe, 0xb9, 0x06, 0x4d,
- 0x41, 0x2a, 0xf2, 0x6c, 0x4c, 0xf2, 0x8c, 0x6e, 0x41, 0x8f, 0xa4, 0xee, 0x24, 0x19, 0x35, 0x81,
- 0xaf, 0x4b, 0xd2, 0x22, 0xef, 0xe8, 0x3d, 0x68, 0xf9, 0x47, 0x59, 0x7c, 0x9c, 0x5a, 0x75, 0xe1,
- 0x6a, 0x61, 0xe2, 0x8a, 0x07, 0xbb, 0xc5, 0x79, 0x8e, 0x12, 0x41, 0x0f, 0x01, 0x3c, 0xc6, 0x12,
- 0x72, 0x98, 0x31, 0x9c, 0x8a, 0x68, 0xbb, 0x0f, 0x2c, 0x4d, 0x21, 0x4b, 0xf1, 0x93, 0x82, 0xef,
- 0x68, 0xb2, 0xe8, 0x11, 0x74, 0xf0, 0x19, 0xc3, 0x71, 0x80, 0x03, 0xab, 0x29, 0x1c, 0xad, 0x4c,
- 0xc5, 0xb4, 0xb9, 0xa3, 0xf8, 0x32, 0xc2, 0x42, 0x7c, 0xf0, 0x18, 0xfa, 0x25, 0x16, 0x9a, 0x83,
- 0xfa, 0x31, 0xce, 0x6f, 0x96, 0x7f, 0xf2, 0xec, 0x9e, 0x78, 0x61, 0x26, 0x8b, 0xac, 0xe7, 0xc8,
- 0xc3, 0x47, 0xb5, 0x87, 0x86, 0xbd, 0x0d, 0xe6, 0x6e, 0x16, 0x86, 0x85, 0x62, 0x40, 0x92, 0x5c,
- 0x31, 0x20, 0xc9, 0xa4, 0xd0, 0x6a, 0x17, 0x16, 0xda, 0xaf, 0x06, 0xcc, 0xef, 0x9c, 0xe0, 0x98,
- 0x3d, 0xa7, 0x8c, 0x0c, 0x89, 0xef, 0x31, 0x42, 0x63, 0x74, 0x0f, 0x4c, 0x1a, 0x06, 0xee, 0x85,
- 0x95, 0xda, 0xa1, 0xa1, 0x42, 0x7d, 0x0f, 0xcc, 0x18, 0x9f, 0xba, 0x17, 0xba, 0xeb, 0xc4, 0xf8,
- 0x54, 0x4a, 0xaf, 0x43, 0x3f, 0xc0, 0x21, 0x66, 0xd8, 0x2d, 0x6e, 0x87, 0x5f, 0x5d, 0x4f, 0x12,
- 0xb7, 0xe4, 0x75, 0xdc, 0x81, 0x6b, 0xdc, 0xe4, 0xd8, 0x4b, 0x70, 0xcc, 0xdc, 0xb1, 0xc7, 0x8e,
- 0xc4, 0x9d, 0x98, 0x4e, 0x3f, 0xc6, 0xa7, 0x2f, 0x05, 0xf5, 0xa5, 0xc7, 0x8e, 0xec, 0xbf, 0x0c,
- 0x30, 0x8b, 0xcb, 0x44, 0x37, 0xa0, 0xcd, 0xdd, 0xba, 0x24, 0x50, 0x99, 0x68, 0xf1, 0xe3, 0x5e,
- 0xc0, 0x3b, 0x83, 0x0e, 0x87, 0x29, 0x66, 0x02, 0x5e, 0xdd, 0x51, 0x27, 0x5e, 0x59, 0x29, 0xf9,
- 0x4e, 0x36, 0x43, 0xc3, 0x11, 0xdf, 0x3c, 0xe3, 0x11, 0x23, 0x11, 0x16, 0x0e, 0xeb, 0x8e, 0x3c,
- 0xa0, 0x05, 0x68, 0x62, 0x97, 0x79, 0x23, 0x51, 0xe5, 0xa6, 0xd3, 0xc0, 0xaf, 0xbc, 0x11, 0xfa,
- 0x1f, 0x5c, 0x4d, 0x69, 0x96, 0xf8, 0xd8, 0xcd, 0xdd, 0xb6, 0x04, 0xb7, 0x27, 0xa9, 0xbb, 0xd2,
- 0xb9, 0x0d, 0xf5, 0x21, 0x09, 0xac, 0xb6, 0x48, 0xcc, 0x5c, 0xb9, 0x08, 0xf7, 0x02, 0x87, 0x33,
- 0xd1, 0xfb, 0x00, 0x85, 0xa5, 0xc0, 0xea, 0x9c, 0x23, 0x6a, 0xe6, 0x76, 0x03, 0xfb, 0x0b, 0x68,
- 0x29, 0xf3, 0x4b, 0x60, 0x9e, 0xd0, 0x30, 0x8b, 0x8a, 0xb0, 0xfb, 0x4e, 0x47, 0x12, 0xf6, 0x02,
- 0x74, 0x13, 0xc4, 0xac, 0x73, 0x79, 0x55, 0xd5, 0x44, 0x90, 0x22, 0x43, 0x9f, 0x61, 0x31, 0x2d,
- 0x7c, 0x4a, 0x8f, 0x89, 0x8c, 0xbe, 0xed, 0xa8, 0x93, 0xfd, 0x67, 0x0d, 0xae, 0x96, 0xcb, 0x9d,
- 0xbb, 0x10, 0x56, 0x44, 0xae, 0x0c, 0x61, 0x46, 0x98, 0x3d, 0x28, 0xe5, 0xab, 0xa6, 0xe7, 0x2b,
- 0x57, 0x89, 0x68, 0x20, 0x1d, 0xf4, 0xa5, 0xca, 0x33, 0x1a, 0x60, 0x5e, 0xad, 0x19, 0x09, 0x44,
- 0x82, 0xfb, 0x0e, 0xff, 0xe4, 0x94, 0x11, 0x09, 0xd4, 0x08, 0xe1, 0x9f, 0x02, 0x5e, 0x22, 0xec,
- 0xb6, 0xe4, 0x95, 0xc9, 0x13, 0xbf, 0xb2, 0x88, 0x53, 0xdb, 0xf2, 0x1e, 0xf8, 0x37, 0x5a, 0x83,
- 0x6e, 0x82, 0xc7, 0xa1, 0xaa, 0x5e, 0x91, 0x3e, 0xd3, 0xd1, 0x49, 0x68, 0x15, 0xc0, 0xa7, 0x61,
- 0x88, 0x7d, 0x21, 0x60, 0x0a, 0x01, 0x8d, 0xc2, 0x2b, 0x87, 0xb1, 0xd0, 0x4d, 0xb1, 0x6f, 0xc1,
- 0x9a, 0xb1, 0xd1, 0x74, 0x5a, 0x8c, 0x85, 0x07, 0xd8, 0xe7, 0x71, 0x64, 0x29, 0x4e, 0x5c, 0x31,
- 0x80, 0xba, 0x42, 0xaf, 0xc3, 0x09, 0x62, 0x54, 0xae, 0x00, 0x8c, 0x12, 0x9a, 0x8d, 0x25, 0xb7,
- 0xb7, 0x56, 0xe7, 0xf3, 0x58, 0x50, 0x04, 0xfb, 0x36, 0x5c, 0x4d, 0xdf, 0x44, 0x21, 0x89, 0x8f,
- 0x5d, 0xe6, 0x25, 0x23, 0xcc, 0xac, 0xbe, 0xac, 0x61, 0x45, 0x7d, 0x25, 0x88, 0xf6, 0x97, 0x80,
- 0xb6, 0x12, 0xec, 0x31, 0xfc, 0x0f, 0x56, 0xcf, 0x5b, 0x76, 0xf7, 0x75, 0x58, 0x28, 0x99, 0x96,
- 0x53, 0x98, 0x7b, 0x7c, 0x3d, 0x0e, 0xde, 0x95, 0xc7, 0x92, 0x69, 0xe5, 0xf1, 0x07, 0x03, 0xd0,
- 0xb6, 0x68, 0xf0, 0x7f, 0xb7, 0x5f, 0x79, 0xcb, 0xf1, 0xb9, 0x2f, 0x07, 0x48, 0xe0, 0x31, 0x4f,
- 0x6d, 0xa6, 0x1e, 0x49, 0xa5, 0xfd, 0x6d, 0x8f, 0x79, 0x6a, 0x3b, 0x24, 0xd8, 0xcf, 0x12, 0xbe,
- 0xac, 0x44, 0x5d, 0x89, 0xed, 0xe0, 0xe4, 0x24, 0x0e, 0xb4, 0x04, 0x48, 0x01, 0xfd, 0xc9, 0x00,
- 0xeb, 0x09, 0xa3, 0x11, 0xf1, 0x1d, 0xcc, 0x1d, 0x96, 0xe0, 0xae, 0x43, 0x9f, 0x8f, 0xc5, 0x69,
- 0xc8, 0x3d, 0x1a, 0x06, 0x93, 0xb5, 0x73, 0x13, 0xf8, 0x64, 0x74, 0x35, 0xe4, 0x6d, 0x1a, 0x06,
- 0xa2, 0x20, 0xd6, 0x81, 0x8f, 0x2f, 0x4d, 0x5f, 0x2e, 0xe1, 0x5e, 0x8c, 0x4f, 0x4b, 0xfa, 0x5c,
- 0x48, 0xe8, 0xcb, 0x99, 0xd7, 0x8e, 0xf1, 0x29, 0xd7, 0xb7, 0x97, 0xe0, 0xe6, 0x0c, 0x6c, 0x0a,
- 0xf9, 0x2f, 0x06, 0x2c, 0x3c, 0x49, 0x53, 0x32, 0x8a, 0x3f, 0x17, 0xdd, 0x9f, 0x83, 0x5e, 0x84,
- 0xa6, 0x4f, 0xb3, 0x98, 0x09, 0xb0, 0x4d, 0x47, 0x1e, 0xa6, 0x1a, 0xa2, 0x56, 0x69, 0x88, 0xa9,
- 0x96, 0xaa, 0x57, 0x5b, 0x4a, 0x6b, 0x99, 0x46, 0xa9, 0x65, 0xfe, 0x0b, 0x5d, 0x7e, 0x31, 0xae,
- 0x8f, 0x63, 0x86, 0x13, 0x35, 0x30, 0x81, 0x93, 0xb6, 0x04, 0xc5, 0xfe, 0xde, 0x80, 0xc5, 0x32,
- 0x52, 0xf5, 0x3a, 0x38, 0x77, 0x7e, 0xf3, 0x81, 0x91, 0x84, 0x0a, 0x26, 0xff, 0xe4, 0xad, 0x37,
- 0xce, 0x0e, 0x43, 0xe2, 0xbb, 0x9c, 0x21, 0xe1, 0x99, 0x92, 0xf2, 0x3a, 0x09, 0x27, 0x41, 0x37,
- 0xf4, 0xa0, 0x11, 0x34, 0xbc, 0x8c, 0x1d, 0xe5, 0x33, 0x9c, 0x7f, 0xdb, 0x1f, 0xc2, 0x82, 0x7c,
- 0xb0, 0x95, 0xb3, 0xb6, 0x02, 0x50, 0x4c, 0x55, 0xf9, 0x56, 0x31, 0x1d, 0x33, 0x1f, 0xab, 0xa9,
- 0xfd, 0x31, 0x98, 0xfb, 0x54, 0x26, 0x22, 0x45, 0xf7, 0xc1, 0x0c, 0xf3, 0x83, 0x7a, 0xd6, 0xa0,
- 0x49, 0x7b, 0xe4, 0x72, 0xce, 0x44, 0xc8, 0x7e, 0x0c, 0x9d, 0x9c, 0x9c, 0xc7, 0x66, 0x9c, 0x17,
- 0x5b, 0x6d, 0x2a, 0x36, 0xfb, 0x77, 0x03, 0x16, 0xcb, 0x90, 0x55, 0xfa, 0x5e, 0x43, 0xbf, 0x70,
- 0xe1, 0x46, 0xde, 0x58, 0x61, 0xb9, 0xaf, 0x63, 0xa9, 0xaa, 0x15, 0x00, 0xd3, 0x67, 0xde, 0x58,
- 0x96, 0x54, 0x2f, 0xd4, 0x48, 0x83, 0x57, 0x30, 0x5f, 0x11, 0x99, 0xf1, 0x52, 0xb9, 0xab, 0xbf,
- 0x54, 0x4a, 0xaf, 0xad, 0x42, 0x5b, 0x7f, 0xbe, 0x3c, 0x82, 0x1b, 0xb2, 0xff, 0xb6, 0x8a, 0xa2,
- 0xcb, 0x73, 0x5f, 0xae, 0x4d, 0x63, 0xba, 0x36, 0xed, 0x01, 0x58, 0x55, 0x55, 0xd5, 0x05, 0x23,
- 0x98, 0x3f, 0x60, 0x1e, 0x23, 0x29, 0x23, 0x7e, 0xf1, 0x6c, 0x9e, 0x2a, 0x66, 0xe3, 0xb2, 0xfd,
- 0x50, 0x6d, 0x87, 0x39, 0xa8, 0x33, 0x96, 0xd7, 0x19, 0xff, 0xe4, 0xb7, 0x80, 0x74, 0x4f, 0xea,
- 0x0e, 0xde, 0x81, 0x2b, 0x5e, 0x0f, 0x8c, 0x32, 0x2f, 0x94, 0xfb, 0xb7, 0x21, 0xf6, 0xaf, 0x29,
- 0x28, 0x62, 0x01, 0xcb, 0x15, 0x15, 0x48, 0x6e, 0x53, 0x6e, 0x67, 0x4e, 0x10, 0xcc, 0x15, 0x00,
- 0xd1, 0x52, 0xb2, 0x1b, 0x5a, 0x52, 0x97, 0x53, 0xb6, 0x38, 0xc1, 0x5e, 0x85, 0xe5, 0x4f, 0x31,
- 0xe3, 0x2f, 0x89, 0x64, 0x8b, 0xc6, 0x43, 0x32, 0xca, 0x12, 0x4f, 0xbb, 0x0a, 0xfb, 0x47, 0x03,
- 0x56, 0xce, 0x11, 0x50, 0x01, 0x5b, 0xd0, 0x8e, 0xbc, 0x94, 0xe1, 0x24, 0xef, 0x92, 0xfc, 0x38,
- 0x9d, 0x8a, 0xda, 0x65, 0xa9, 0xa8, 0x57, 0x52, 0x71, 0x1d, 0x5a, 0x91, 0x77, 0xe6, 0x46, 0x87,
- 0xea, 0xa9, 0xd0, 0x8c, 0xbc, 0xb3, 0x67, 0x87, 0x0f, 0xfe, 0x68, 0x43, 0xef, 0x00, 0x7b, 0xa7,
- 0x18, 0x07, 0x02, 0x18, 0x1a, 0xe5, 0x0d, 0x51, 0xfe, 0xd1, 0x85, 0x6e, 0x4f, 0x57, 0xfe, 0xcc,
- 0x5f, 0x79, 0x83, 0x3b, 0x97, 0x89, 0xa9, 0xda, 0xba, 0x82, 0xf6, 0xa1, 0xab, 0xfd, 0xaa, 0x41,
- 0xcb, 0x9a, 0x62, 0xe5, 0xc7, 0xda, 0x60, 0xe5, 0x1c, 0xae, 0x6e, 0x4d, 0xdb, 0xce, 0xba, 0xb5,
- 0xea, 0x7b, 0x40, 0xb7, 0x36, 0x6b, 0xa5, 0x0b, 0x6b, 0xda, 0xe6, 0xd5, 0xad, 0x55, 0x77, 0xbd,
- 0x6e, 0x6d, 0xd6, 0xba, 0x16, 0xd6, 0xb4, 0xf5, 0xa8, 0x5b, 0xab, 0xae, 0x71, 0xdd, 0xda, 0xac,
- 0x9d, 0x7a, 0x05, 0x7d, 0x0d, 0xf3, 0x95, 0xc5, 0x85, 0xec, 0x89, 0xd6, 0x79, 0x1b, 0x77, 0xb0,
- 0x7e, 0xa1, 0x4c, 0x61, 0xff, 0x05, 0xf4, 0xf4, 0x85, 0x82, 0x34, 0x40, 0x33, 0x56, 0xe2, 0x60,
- 0xf5, 0x3c, 0xb6, 0x6e, 0x50, 0x9f, 0x95, 0xba, 0xc1, 0x19, 0xdb, 0x42, 0x37, 0x38, 0x6b, 0xc4,
- 0xda, 0x57, 0xd0, 0x57, 0x30, 0x37, 0x3d, 0xb3, 0xd0, 0xad, 0xe9, 0xb4, 0x55, 0x46, 0xe1, 0xc0,
- 0xbe, 0x48, 0xa4, 0x30, 0xbe, 0x07, 0x30, 0x19, 0x45, 0x68, 0x69, 0xa2, 0x53, 0x19, 0x85, 0x83,
- 0xe5, 0xd9, 0xcc, 0xc2, 0xd4, 0x37, 0x70, 0x7d, 0x66, 0xbf, 0x23, 0xad, 0x49, 0x2e, 0x9a, 0x18,
- 0x83, 0xff, 0x5f, 0x2a, 0x97, 0xfb, 0x7a, 0xba, 0x0a, 0x73, 0xa9, 0x6c, 0xe3, 0x61, 0xba, 0xe9,
- 0x87, 0x04, 0xc7, 0xec, 0x29, 0x08, 0x8d, 0x97, 0x09, 0x65, 0xf4, 0xb0, 0x25, 0xfe, 0xad, 0xf9,
- 0xe0, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x2d, 0xec, 0xb0, 0x56, 0xbc, 0x11, 0x00, 0x00,
+ // 1603 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x58, 0xcd, 0x6f, 0xdc, 0x44,
+ 0x14, 0x8f, 0xf7, 0xdb, 0x6f, 0x77, 0xdb, 0x64, 0x92, 0xb6, 0xdb, 0xcd, 0x07, 0xa9, 0x43, 0x4b,
+ 0x10, 0x55, 0xa8, 0x42, 0x0f, 0x2d, 0x85, 0x43, 0x9b, 0x0f, 0x14, 0x91, 0x7e, 0xc8, 0x69, 0x11,
+ 0x08, 0x09, 0xcb, 0xb1, 0x67, 0x37, 0x43, 0x6c, 0xcf, 0x32, 0x1e, 0x27, 0x29, 0x7f, 0x02, 0x47,
+ 0x8e, 0x48, 0x9c, 0xf9, 0x27, 0x10, 0x17, 0x84, 0xf8, 0x6f, 0x38, 0x72, 0x46, 0x33, 0x63, 0x7b,
+ 0xc7, 0xeb, 0x4d, 0xd2, 0x0a, 0xf5, 0xe6, 0x79, 0xdf, 0xef, 0xcd, 0x7b, 0xbf, 0x37, 0xbb, 0xd0,
+ 0x1e, 0x90, 0x00, 0xb3, 0x8d, 0x11, 0xa3, 0x9c, 0xa2, 0x96, 0x3c, 0x38, 0xa3, 0x43, 0xeb, 0x39,
+ 0x2c, 0xee, 0x53, 0x7a, 0x9c, 0x8c, 0xb6, 0x09, 0xc3, 0x1e, 0xa7, 0xec, 0xf5, 0x4e, 0xc4, 0xd9,
+ 0x6b, 0x1b, 0xff, 0x90, 0xe0, 0x98, 0xa3, 0x25, 0x30, 0xfd, 0x8c, 0xd1, 0x33, 0x56, 0x8d, 0x75,
+ 0xd3, 0x1e, 0x13, 0x10, 0x82, 0x5a, 0xe4, 0x86, 0xb8, 0x57, 0x91, 0x0c, 0xf9, 0x6d, 0xed, 0xc0,
+ 0xd2, 0x74, 0x83, 0xf1, 0x88, 0x46, 0x31, 0x46, 0xb7, 0xa1, 0x8e, 0x05, 0x41, 0x5a, 0x6b, 0x6f,
+ 0x5e, 0xdd, 0xc8, 0x42, 0xd9, 0x50, 0x72, 0x8a, 0x6b, 0xfd, 0x61, 0x00, 0xda, 0x27, 0x31, 0x17,
+ 0x44, 0x82, 0xe3, 0x37, 0x8b, 0xe7, 0x3a, 0x34, 0x46, 0x0c, 0x0f, 0xc8, 0x59, 0x1a, 0x51, 0x7a,
+ 0x42, 0x77, 0x61, 0x2e, 0xe6, 0x2e, 0xe3, 0xbb, 0x8c, 0x86, 0xbb, 0x24, 0xc0, 0xcf, 0x44, 0xd0,
+ 0x55, 0x29, 0x52, 0x66, 0xa0, 0x0d, 0x40, 0x24, 0xf2, 0x82, 0x24, 0x26, 0x27, 0xf8, 0x20, 0xe3,
+ 0xf6, 0x6a, 0xab, 0xc6, 0x7a, 0xcb, 0x9e, 0xc2, 0x41, 0x0b, 0x50, 0x0f, 0x48, 0x48, 0x78, 0xaf,
+ 0xbe, 0x6a, 0xac, 0x77, 0x6d, 0x75, 0xb0, 0x3e, 0x83, 0xf9, 0x42, 0xfc, 0x6f, 0x97, 0xfe, 0xaf,
+ 0x15, 0xa8, 0x4b, 0x42, 0x5e, 0x63, 0x63, 0x5c, 0x63, 0x74, 0x0b, 0x3a, 0x24, 0x76, 0xc6, 0x85,
+ 0xa8, 0xc8, 0xd8, 0xda, 0x24, 0xce, 0x6b, 0x8e, 0x3e, 0x82, 0x86, 0x77, 0x94, 0x44, 0xc7, 0x71,
+ 0xaf, 0xba, 0x5a, 0x5d, 0x6f, 0x6f, 0xce, 0x8f, 0x1d, 0x89, 0x44, 0xb7, 0x04, 0xcf, 0x4e, 0x45,
+ 0xd0, 0x03, 0x00, 0x97, 0x73, 0x46, 0x0e, 0x13, 0x8e, 0x63, 0x99, 0x69, 0x7b, 0xb3, 0xa7, 0x29,
+ 0x24, 0x31, 0x7e, 0x9c, 0xf3, 0x6d, 0x4d, 0x16, 0x3d, 0x84, 0x16, 0x3e, 0xe3, 0x38, 0xf2, 0xb1,
+ 0xdf, 0xab, 0x4b, 0x47, 0xcb, 0x13, 0x19, 0x6d, 0xec, 0xa4, 0x7c, 0x95, 0x5f, 0x2e, 0xde, 0x7f,
+ 0x04, 0xdd, 0x02, 0x0b, 0xcd, 0x42, 0xf5, 0x18, 0x67, 0xb7, 0x2a, 0x3e, 0x45, 0x65, 0x4f, 0xdc,
+ 0x20, 0x51, 0x0d, 0xd6, 0xb1, 0xd5, 0xe1, 0xd3, 0xca, 0x03, 0xc3, 0xda, 0x06, 0x73, 0x37, 0x09,
+ 0x82, 0x5c, 0xd1, 0x27, 0x2c, 0x53, 0xf4, 0x09, 0x1b, 0x57, 0xb9, 0x72, 0x61, 0x95, 0x7f, 0x37,
+ 0x60, 0x6e, 0xe7, 0x04, 0x47, 0xfc, 0x19, 0xe5, 0x64, 0x40, 0x3c, 0x97, 0x13, 0x1a, 0xa1, 0xbb,
+ 0x60, 0xd2, 0xc0, 0x77, 0x2e, 0xbc, 0xa6, 0x16, 0x0d, 0xd2, 0xa8, 0xef, 0x82, 0x19, 0xe1, 0x53,
+ 0xe7, 0x42, 0x77, 0xad, 0x08, 0x9f, 0x2a, 0xe9, 0x35, 0xe8, 0xfa, 0x38, 0xc0, 0x1c, 0x3b, 0xf9,
+ 0xed, 0x88, 0xab, 0xeb, 0x28, 0xe2, 0x96, 0xba, 0x8e, 0x3b, 0x70, 0x55, 0x98, 0x1c, 0xb9, 0x0c,
+ 0x47, 0xdc, 0x19, 0xb9, 0xfc, 0x48, 0xde, 0x89, 0x69, 0x77, 0x23, 0x7c, 0xfa, 0x42, 0x52, 0x5f,
+ 0xb8, 0xfc, 0xc8, 0xfa, 0xd7, 0x00, 0x33, 0xbf, 0x4c, 0x74, 0x03, 0x9a, 0xc2, 0xad, 0x43, 0xfc,
+ 0xb4, 0x12, 0x0d, 0x71, 0xdc, 0xf3, 0xc5, 0x54, 0xd0, 0xc1, 0x20, 0xc6, 0x5c, 0x86, 0x57, 0xb5,
+ 0xd3, 0x93, 0xe8, 0xac, 0x98, 0xfc, 0xa8, 0x06, 0xa1, 0x66, 0xcb, 0x6f, 0x51, 0xf1, 0x90, 0x93,
+ 0x10, 0x4b, 0x87, 0x55, 0x5b, 0x1d, 0xd0, 0x3c, 0xd4, 0xb1, 0xc3, 0xdd, 0xa1, 0xec, 0x70, 0xd3,
+ 0xae, 0xe1, 0x97, 0xee, 0x10, 0xbd, 0x0f, 0x57, 0x62, 0x9a, 0x30, 0x0f, 0x3b, 0x99, 0xdb, 0x86,
+ 0xe4, 0x76, 0x14, 0x75, 0x57, 0x39, 0xb7, 0xa0, 0x3a, 0x20, 0x7e, 0xaf, 0x29, 0x0b, 0x33, 0x5b,
+ 0x6c, 0xc2, 0x3d, 0xdf, 0x16, 0x4c, 0xf4, 0x31, 0x40, 0x6e, 0xc9, 0xef, 0xb5, 0xce, 0x11, 0x35,
+ 0x33, 0xbb, 0xbe, 0xf5, 0x35, 0x34, 0x52, 0xf3, 0x8b, 0x60, 0x9e, 0xd0, 0x20, 0x09, 0xf3, 0xb4,
+ 0xbb, 0x76, 0x4b, 0x11, 0xf6, 0x7c, 0x74, 0x13, 0x24, 0xce, 0x39, 0xa2, 0xab, 0x2a, 0x32, 0x49,
+ 0x59, 0xa1, 0x2f, 0xb1, 0x44, 0x0a, 0x8f, 0xd2, 0x63, 0xa2, 0xb2, 0x6f, 0xda, 0xe9, 0xc9, 0xfa,
+ 0xa7, 0x02, 0x57, 0x8a, 0xed, 0x2e, 0x5c, 0x48, 0x2b, 0xb2, 0x56, 0x86, 0x34, 0x23, 0xcd, 0x1e,
+ 0x14, 0xea, 0x55, 0xd1, 0xeb, 0x95, 0xa9, 0x84, 0xd4, 0x57, 0x0e, 0xba, 0x4a, 0xe5, 0x29, 0xf5,
+ 0xb1, 0xe8, 0xd6, 0x84, 0xf8, 0xb2, 0xc0, 0x5d, 0x5b, 0x7c, 0x0a, 0xca, 0x90, 0xf8, 0x29, 0x7c,
+ 0x88, 0x4f, 0x19, 0x1e, 0x93, 0x76, 0x1b, 0xea, 0xca, 0xd4, 0x49, 0x5c, 0x59, 0x28, 0xa8, 0x4d,
+ 0x75, 0x0f, 0xe2, 0x1b, 0xad, 0x42, 0x9b, 0xe1, 0x51, 0x90, 0x76, 0xaf, 0x2c, 0x9f, 0x69, 0xeb,
+ 0x24, 0xb4, 0x02, 0xe0, 0xd1, 0x20, 0xc0, 0x9e, 0x14, 0x30, 0xa5, 0x80, 0x46, 0x11, 0x9d, 0xc3,
+ 0x79, 0xe0, 0xc4, 0xd8, 0xeb, 0xc1, 0xaa, 0xb1, 0x5e, 0xb7, 0x1b, 0x9c, 0x07, 0x07, 0xd8, 0x13,
+ 0x79, 0x24, 0x31, 0x66, 0x8e, 0x04, 0xa0, 0xb6, 0xd4, 0x6b, 0x09, 0x82, 0x84, 0xc9, 0x65, 0x80,
+ 0x21, 0xa3, 0xc9, 0x48, 0x71, 0x3b, 0xab, 0x55, 0x81, 0xc5, 0x92, 0x22, 0xd9, 0xb7, 0xe1, 0x4a,
+ 0xfc, 0x3a, 0x0c, 0x48, 0x74, 0xec, 0x70, 0x97, 0x0d, 0x31, 0xef, 0x75, 0x55, 0x0f, 0xa7, 0xd4,
+ 0x97, 0x92, 0x68, 0x7d, 0x03, 0x68, 0x8b, 0x61, 0x97, 0xe3, 0xb7, 0x58, 0x3b, 0x6f, 0x38, 0xdd,
+ 0xd7, 0x60, 0xbe, 0x60, 0x5a, 0x21, 0xb0, 0xf0, 0xf8, 0x6a, 0xe4, 0xbf, 0x2b, 0x8f, 0x05, 0xd3,
+ 0xa9, 0xc7, 0xbf, 0x0c, 0x40, 0xdb, 0x72, 0xc0, 0xff, 0xdf, 0x6e, 0x15, 0x23, 0x27, 0x70, 0x5f,
+ 0x01, 0x88, 0xef, 0x72, 0x37, 0xdd, 0x4a, 0x1d, 0x12, 0x2b, 0xfb, 0xdb, 0x2e, 0x77, 0xd3, 0xed,
+ 0xc0, 0xb0, 0x97, 0x30, 0xb1, 0xa8, 0x64, 0x5f, 0xc9, 0xed, 0x60, 0x67, 0x24, 0x74, 0x1f, 0xae,
+ 0x93, 0x61, 0x44, 0x19, 0x1e, 0x8b, 0x39, 0x98, 0x31, 0xca, 0x64, 0xbf, 0xb5, 0xec, 0x05, 0xc5,
+ 0xcd, 0x15, 0x76, 0x04, 0x4f, 0xa4, 0x57, 0x48, 0x23, 0x4d, 0xef, 0x17, 0x03, 0x7a, 0x8f, 0x39,
+ 0x0d, 0x89, 0x67, 0x63, 0x11, 0x66, 0x21, 0xc9, 0x35, 0xe8, 0x0a, 0x30, 0x9d, 0x4c, 0xb4, 0x43,
+ 0x03, 0x7f, 0xbc, 0xac, 0x6e, 0x82, 0xc0, 0x53, 0x47, 0xcb, 0xb7, 0x49, 0x03, 0x5f, 0xb6, 0xd1,
+ 0x1a, 0x08, 0xd0, 0xd3, 0xf4, 0xd5, 0xda, 0xee, 0x44, 0xf8, 0xb4, 0xa0, 0x2f, 0x84, 0xa4, 0xbe,
+ 0x42, 0xca, 0x66, 0x84, 0x4f, 0x85, 0xbe, 0xb5, 0x08, 0x37, 0xa7, 0xc4, 0x96, 0x46, 0xfe, 0x9b,
+ 0x01, 0xf3, 0x8f, 0xe3, 0x98, 0x0c, 0xa3, 0xaf, 0x24, 0x66, 0x64, 0x41, 0x2f, 0x40, 0xdd, 0xa3,
+ 0x49, 0xc4, 0x65, 0xb0, 0x75, 0x5b, 0x1d, 0x26, 0xc6, 0xa8, 0x52, 0x1a, 0xa3, 0x89, 0x41, 0xac,
+ 0x96, 0x07, 0x51, 0x1b, 0xb4, 0x5a, 0x61, 0xd0, 0xde, 0x83, 0xb6, 0xb8, 0x4e, 0xc7, 0xc3, 0x11,
+ 0xc7, 0x2c, 0x85, 0x59, 0x10, 0xa4, 0x2d, 0x49, 0xb1, 0x7e, 0x32, 0x60, 0xa1, 0x18, 0x69, 0xfa,
+ 0x9e, 0x38, 0x17, 0xf5, 0x05, 0xcc, 0xb0, 0x20, 0x0d, 0x53, 0x7c, 0x8a, 0x81, 0x1d, 0x25, 0x87,
+ 0x01, 0xf1, 0x1c, 0xc1, 0x50, 0xe1, 0x99, 0x8a, 0xf2, 0x8a, 0x05, 0xe3, 0xa4, 0x6b, 0x7a, 0xd2,
+ 0x08, 0x6a, 0x6e, 0xc2, 0x8f, 0x32, 0xe4, 0x17, 0xdf, 0xd6, 0x7d, 0x98, 0x57, 0x4f, 0xbc, 0x62,
+ 0xd5, 0x96, 0x01, 0x72, 0x2c, 0x8e, 0x7b, 0x86, 0x02, 0x84, 0x0c, 0x8c, 0x63, 0xeb, 0x73, 0x30,
+ 0xf7, 0xa9, 0x2a, 0x44, 0x8c, 0xee, 0x81, 0x19, 0x64, 0x07, 0x29, 0xda, 0xde, 0x44, 0xe3, 0xa1,
+ 0xca, 0xe4, 0xec, 0xb1, 0x90, 0xf5, 0x08, 0x5a, 0x19, 0x39, 0xcb, 0xcd, 0x38, 0x2f, 0xb7, 0xca,
+ 0x44, 0x6e, 0xd6, 0x9f, 0x06, 0x2c, 0x14, 0x43, 0x4e, 0xcb, 0xf7, 0x0a, 0xba, 0xb9, 0x0b, 0x27,
+ 0x74, 0x47, 0x69, 0x2c, 0xf7, 0xf4, 0x58, 0xca, 0x6a, 0x79, 0x80, 0xf1, 0x53, 0x77, 0xa4, 0x5a,
+ 0xaa, 0x13, 0x68, 0xa4, 0xfe, 0x4b, 0x98, 0x2b, 0x89, 0x4c, 0x79, 0xdf, 0x7c, 0xa8, 0xbf, 0x6f,
+ 0x0a, 0x6f, 0xb4, 0x5c, 0x5b, 0x7f, 0xf4, 0x3c, 0x84, 0x1b, 0x6a, 0xfe, 0xb6, 0xf2, 0xa6, 0xcb,
+ 0x6a, 0x5f, 0xec, 0x4d, 0x63, 0xb2, 0x37, 0xad, 0x3e, 0xf4, 0xca, 0xaa, 0xe9, 0x14, 0x0c, 0x61,
+ 0xee, 0x80, 0xbb, 0x9c, 0xc4, 0x9c, 0x78, 0xf9, 0x43, 0x7b, 0xa2, 0x99, 0x8d, 0xcb, 0xb6, 0x4a,
+ 0x79, 0x1c, 0x66, 0xa1, 0xca, 0x79, 0xd6, 0x67, 0xe2, 0x53, 0xdc, 0x02, 0xd2, 0x3d, 0xa5, 0x77,
+ 0xf0, 0x0e, 0x5c, 0x89, 0x7e, 0xe0, 0x94, 0xbb, 0x81, 0xda, 0xda, 0x35, 0xb9, 0xb5, 0x4d, 0x49,
+ 0x91, 0x6b, 0x5b, 0x2d, 0x36, 0x5f, 0x71, 0xeb, 0x6a, 0xa7, 0x0b, 0x82, 0x64, 0x2e, 0x03, 0xc8,
+ 0x91, 0x52, 0xd3, 0xd0, 0x50, 0xba, 0x82, 0xb2, 0x25, 0x08, 0xd6, 0x0a, 0x2c, 0x7d, 0x81, 0xb9,
+ 0x78, 0x7f, 0xb0, 0x2d, 0x1a, 0x0d, 0xc8, 0x30, 0x61, 0xae, 0x76, 0x15, 0xd6, 0xcf, 0x06, 0x2c,
+ 0x9f, 0x23, 0x90, 0x26, 0xdc, 0x83, 0x66, 0xe8, 0xc6, 0x1c, 0xb3, 0x6c, 0x4a, 0xb2, 0xe3, 0x64,
+ 0x29, 0x2a, 0x97, 0x95, 0xa2, 0x5a, 0x2a, 0xc5, 0x35, 0x68, 0x84, 0xee, 0x99, 0x13, 0x1e, 0xa6,
+ 0x0f, 0x8c, 0x7a, 0xe8, 0x9e, 0x3d, 0x3d, 0xdc, 0xfc, 0xbb, 0x09, 0x9d, 0x03, 0xec, 0x9e, 0x62,
+ 0xec, 0xcb, 0xc0, 0xd0, 0x30, 0x1b, 0x88, 0xe2, 0xcf, 0x34, 0x74, 0x7b, 0xb2, 0xf3, 0xa7, 0xfe,
+ 0x2e, 0xec, 0xdf, 0xb9, 0x4c, 0x2c, 0xed, 0xad, 0x19, 0xf4, 0x0c, 0xda, 0xda, 0xef, 0x20, 0xb4,
+ 0xa4, 0x29, 0x96, 0x7e, 0xde, 0xf5, 0x97, 0xcf, 0xe1, 0x66, 0xd6, 0xee, 0x19, 0x68, 0x1f, 0xda,
+ 0xda, 0x56, 0xd7, 0xed, 0x95, 0xdf, 0x11, 0xba, 0xbd, 0x69, 0x4f, 0x81, 0x19, 0x61, 0x4d, 0xdb,
+ 0xd8, 0xba, 0xb5, 0xf2, 0x1b, 0x41, 0xb7, 0x36, 0x6d, 0xcd, 0x4b, 0x6b, 0xda, 0x82, 0xd4, 0xad,
+ 0x95, 0xd7, 0xbf, 0x6e, 0x6d, 0xda, 0x56, 0x9d, 0x41, 0xdf, 0xc1, 0x5c, 0x69, 0x75, 0x21, 0x6b,
+ 0xac, 0x75, 0xde, 0xce, 0xed, 0xaf, 0x5d, 0x28, 0x93, 0xdb, 0x7f, 0x0e, 0x1d, 0x7d, 0xa5, 0x20,
+ 0x2d, 0xa0, 0x29, 0x4b, 0xb1, 0xbf, 0x72, 0x1e, 0x5b, 0x37, 0xa8, 0xa3, 0xa5, 0x6e, 0x70, 0xca,
+ 0xbe, 0xd0, 0x0d, 0x4e, 0x03, 0x59, 0x6b, 0x06, 0x7d, 0x0b, 0xb3, 0x93, 0xa8, 0x85, 0x6e, 0x4d,
+ 0x96, 0xad, 0x04, 0x86, 0x7d, 0xeb, 0x22, 0x91, 0xdc, 0xf8, 0x1e, 0xc0, 0x18, 0x8c, 0xd0, 0xe2,
+ 0x58, 0xa7, 0x04, 0x86, 0xfd, 0xa5, 0xe9, 0xcc, 0xdc, 0xd4, 0xf7, 0x70, 0x6d, 0xea, 0xc4, 0x23,
+ 0x6d, 0x4c, 0x2e, 0xc2, 0x8c, 0xfe, 0x07, 0x97, 0xca, 0x65, 0xbe, 0x9e, 0xac, 0xc0, 0x6c, 0xac,
+ 0x06, 0x79, 0x10, 0x6f, 0x78, 0x01, 0xc1, 0x11, 0x7f, 0x02, 0x52, 0xe3, 0x05, 0xa3, 0x9c, 0x1e,
+ 0x36, 0xe4, 0x3f, 0x3c, 0x9f, 0xfc, 0x17, 0x00, 0x00, 0xff, 0xff, 0x14, 0x43, 0x9d, 0xb9, 0xf0,
+ 0x11, 0x00, 0x00,
}
diff --git a/weed/pb/master.proto b/weed/pb/master.proto
index 7dcab40db..9b1e884c7 100644
--- a/weed/pb/master.proto
+++ b/weed/pb/master.proto
@@ -7,7 +7,7 @@ package master_pb;
service Seaweed {
rpc SendHeartbeat (stream Heartbeat) returns (stream HeartbeatResponse) {
}
- rpc KeepConnected (stream ClientListenRequest) returns (stream VolumeLocation) {
+ rpc KeepConnected (stream KeepConnectedRequest) returns (stream VolumeLocation) {
}
rpc LookupVolume (LookupVolumeRequest) returns (LookupVolumeResponse) {
}
@@ -58,6 +58,7 @@ message HeartbeatResponse {
string leader = 2;
string metrics_address = 3;
uint32 metrics_interval_seconds = 4;
+ repeated StorageBackend storage_backends = 5;
}
message VolumeInformationMessage {
@@ -73,6 +74,8 @@ message VolumeInformationMessage {
uint32 ttl = 10;
uint32 compact_revision = 11;
int64 modified_at_second = 12;
+ string remote_storage_name = 13;
+ string remote_storage_key = 14;
}
message VolumeShortInformationMessage {
@@ -89,6 +92,12 @@ message VolumeEcShardInformationMessage {
uint32 ec_index_bits = 3;
}
+message StorageBackend {
+ string type = 1;
+ string id = 2;
+ map properties = 3;
+}
+
message Empty {
}
@@ -101,7 +110,7 @@ message SuperBlockExtra {
ErasureCoding erasure_coding = 1;
}
-message ClientListenRequest {
+message KeepConnectedRequest {
string name = 1;
}
@@ -110,6 +119,7 @@ message VolumeLocation {
string public_url = 2;
repeated uint32 new_vids = 3;
repeated uint32 deleted_vids = 4;
+ string leader = 5; // optional when leader is not itself
}
message LookupVolumeRequest {
@@ -138,6 +148,8 @@ message AssignRequest {
string data_center = 5;
string rack = 6;
string data_node = 7;
+ uint32 memory_map_max_size_mb = 8;
+ uint32 Writable_volume_count = 9;
}
message AssignResponse {
string fid = 1;
@@ -198,6 +210,7 @@ message DataNodeInfo {
uint64 active_volume_count = 5;
repeated VolumeInformationMessage volume_infos = 6;
repeated VolumeEcShardInformationMessage ec_shard_infos = 7;
+ uint64 remote_volume_count = 8;
}
message RackInfo {
string id = 1;
@@ -206,6 +219,7 @@ message RackInfo {
uint64 free_volume_count = 4;
uint64 active_volume_count = 5;
repeated DataNodeInfo data_node_infos = 6;
+ uint64 remote_volume_count = 7;
}
message DataCenterInfo {
string id = 1;
@@ -214,6 +228,7 @@ message DataCenterInfo {
uint64 free_volume_count = 4;
uint64 active_volume_count = 5;
repeated RackInfo rack_infos = 6;
+ uint64 remote_volume_count = 7;
}
message TopologyInfo {
string id = 1;
@@ -222,6 +237,7 @@ message TopologyInfo {
uint64 free_volume_count = 4;
uint64 active_volume_count = 5;
repeated DataCenterInfo data_center_infos = 6;
+ uint64 remote_volume_count = 7;
}
message VolumeListRequest {
}
diff --git a/weed/pb/master_pb/master.pb.go b/weed/pb/master_pb/master.pb.go
index cc1766fe8..ea4362c92 100644
--- a/weed/pb/master_pb/master.pb.go
+++ b/weed/pb/master_pb/master.pb.go
@@ -14,9 +14,10 @@ It has these top-level messages:
VolumeInformationMessage
VolumeShortInformationMessage
VolumeEcShardInformationMessage
+ StorageBackend
Empty
SuperBlockExtra
- ClientListenRequest
+ KeepConnectedRequest
VolumeLocation
LookupVolumeRequest
LookupVolumeResponse
@@ -204,10 +205,11 @@ func (m *Heartbeat) GetHasNoEcShards() bool {
}
type HeartbeatResponse struct {
- VolumeSizeLimit uint64 `protobuf:"varint,1,opt,name=volume_size_limit,json=volumeSizeLimit" json:"volume_size_limit,omitempty"`
- Leader string `protobuf:"bytes,2,opt,name=leader" json:"leader,omitempty"`
- MetricsAddress string `protobuf:"bytes,3,opt,name=metrics_address,json=metricsAddress" json:"metrics_address,omitempty"`
- MetricsIntervalSeconds uint32 `protobuf:"varint,4,opt,name=metrics_interval_seconds,json=metricsIntervalSeconds" json:"metrics_interval_seconds,omitempty"`
+ VolumeSizeLimit uint64 `protobuf:"varint,1,opt,name=volume_size_limit,json=volumeSizeLimit" json:"volume_size_limit,omitempty"`
+ Leader string `protobuf:"bytes,2,opt,name=leader" json:"leader,omitempty"`
+ MetricsAddress string `protobuf:"bytes,3,opt,name=metrics_address,json=metricsAddress" json:"metrics_address,omitempty"`
+ MetricsIntervalSeconds uint32 `protobuf:"varint,4,opt,name=metrics_interval_seconds,json=metricsIntervalSeconds" json:"metrics_interval_seconds,omitempty"`
+ StorageBackends []*StorageBackend `protobuf:"bytes,5,rep,name=storage_backends,json=storageBackends" json:"storage_backends,omitempty"`
}
func (m *HeartbeatResponse) Reset() { *m = HeartbeatResponse{} }
@@ -243,19 +245,28 @@ func (m *HeartbeatResponse) GetMetricsIntervalSeconds() uint32 {
return 0
}
+func (m *HeartbeatResponse) GetStorageBackends() []*StorageBackend {
+ if m != nil {
+ return m.StorageBackends
+ }
+ return nil
+}
+
type VolumeInformationMessage struct {
- Id uint32 `protobuf:"varint,1,opt,name=id" json:"id,omitempty"`
- Size uint64 `protobuf:"varint,2,opt,name=size" json:"size,omitempty"`
- Collection string `protobuf:"bytes,3,opt,name=collection" json:"collection,omitempty"`
- FileCount uint64 `protobuf:"varint,4,opt,name=file_count,json=fileCount" json:"file_count,omitempty"`
- DeleteCount uint64 `protobuf:"varint,5,opt,name=delete_count,json=deleteCount" json:"delete_count,omitempty"`
- DeletedByteCount uint64 `protobuf:"varint,6,opt,name=deleted_byte_count,json=deletedByteCount" json:"deleted_byte_count,omitempty"`
- ReadOnly bool `protobuf:"varint,7,opt,name=read_only,json=readOnly" json:"read_only,omitempty"`
- ReplicaPlacement uint32 `protobuf:"varint,8,opt,name=replica_placement,json=replicaPlacement" json:"replica_placement,omitempty"`
- Version uint32 `protobuf:"varint,9,opt,name=version" json:"version,omitempty"`
- Ttl uint32 `protobuf:"varint,10,opt,name=ttl" json:"ttl,omitempty"`
- CompactRevision uint32 `protobuf:"varint,11,opt,name=compact_revision,json=compactRevision" json:"compact_revision,omitempty"`
- ModifiedAtSecond int64 `protobuf:"varint,12,opt,name=modified_at_second,json=modifiedAtSecond" json:"modified_at_second,omitempty"`
+ Id uint32 `protobuf:"varint,1,opt,name=id" json:"id,omitempty"`
+ Size uint64 `protobuf:"varint,2,opt,name=size" json:"size,omitempty"`
+ Collection string `protobuf:"bytes,3,opt,name=collection" json:"collection,omitempty"`
+ FileCount uint64 `protobuf:"varint,4,opt,name=file_count,json=fileCount" json:"file_count,omitempty"`
+ DeleteCount uint64 `protobuf:"varint,5,opt,name=delete_count,json=deleteCount" json:"delete_count,omitempty"`
+ DeletedByteCount uint64 `protobuf:"varint,6,opt,name=deleted_byte_count,json=deletedByteCount" json:"deleted_byte_count,omitempty"`
+ ReadOnly bool `protobuf:"varint,7,opt,name=read_only,json=readOnly" json:"read_only,omitempty"`
+ ReplicaPlacement uint32 `protobuf:"varint,8,opt,name=replica_placement,json=replicaPlacement" json:"replica_placement,omitempty"`
+ Version uint32 `protobuf:"varint,9,opt,name=version" json:"version,omitempty"`
+ Ttl uint32 `protobuf:"varint,10,opt,name=ttl" json:"ttl,omitempty"`
+ CompactRevision uint32 `protobuf:"varint,11,opt,name=compact_revision,json=compactRevision" json:"compact_revision,omitempty"`
+ ModifiedAtSecond int64 `protobuf:"varint,12,opt,name=modified_at_second,json=modifiedAtSecond" json:"modified_at_second,omitempty"`
+ RemoteStorageName string `protobuf:"bytes,13,opt,name=remote_storage_name,json=remoteStorageName" json:"remote_storage_name,omitempty"`
+ RemoteStorageKey string `protobuf:"bytes,14,opt,name=remote_storage_key,json=remoteStorageKey" json:"remote_storage_key,omitempty"`
}
func (m *VolumeInformationMessage) Reset() { *m = VolumeInformationMessage{} }
@@ -347,6 +358,20 @@ func (m *VolumeInformationMessage) GetModifiedAtSecond() int64 {
return 0
}
+func (m *VolumeInformationMessage) GetRemoteStorageName() string {
+ if m != nil {
+ return m.RemoteStorageName
+ }
+ return ""
+}
+
+func (m *VolumeInformationMessage) GetRemoteStorageKey() string {
+ if m != nil {
+ return m.RemoteStorageKey
+ }
+ return ""
+}
+
type VolumeShortInformationMessage struct {
Id uint32 `protobuf:"varint,1,opt,name=id" json:"id,omitempty"`
Collection string `protobuf:"bytes,3,opt,name=collection" json:"collection,omitempty"`
@@ -427,13 +452,45 @@ func (m *VolumeEcShardInformationMessage) GetEcIndexBits() uint32 {
return 0
}
+type StorageBackend struct {
+ Type string `protobuf:"bytes,1,opt,name=type" json:"type,omitempty"`
+ Id string `protobuf:"bytes,2,opt,name=id" json:"id,omitempty"`
+ Properties map[string]string `protobuf:"bytes,3,rep,name=properties" json:"properties,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+}
+
+func (m *StorageBackend) Reset() { *m = StorageBackend{} }
+func (m *StorageBackend) String() string { return proto.CompactTextString(m) }
+func (*StorageBackend) ProtoMessage() {}
+func (*StorageBackend) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
+
+func (m *StorageBackend) GetType() string {
+ if m != nil {
+ return m.Type
+ }
+ return ""
+}
+
+func (m *StorageBackend) GetId() string {
+ if m != nil {
+ return m.Id
+ }
+ return ""
+}
+
+func (m *StorageBackend) GetProperties() map[string]string {
+ if m != nil {
+ return m.Properties
+ }
+ return nil
+}
+
type Empty struct {
}
func (m *Empty) Reset() { *m = Empty{} }
func (m *Empty) String() string { return proto.CompactTextString(m) }
func (*Empty) ProtoMessage() {}
-func (*Empty) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
+func (*Empty) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
type SuperBlockExtra struct {
ErasureCoding *SuperBlockExtra_ErasureCoding `protobuf:"bytes,1,opt,name=erasure_coding,json=erasureCoding" json:"erasure_coding,omitempty"`
@@ -442,7 +499,7 @@ type SuperBlockExtra struct {
func (m *SuperBlockExtra) Reset() { *m = SuperBlockExtra{} }
func (m *SuperBlockExtra) String() string { return proto.CompactTextString(m) }
func (*SuperBlockExtra) ProtoMessage() {}
-func (*SuperBlockExtra) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
+func (*SuperBlockExtra) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
func (m *SuperBlockExtra) GetErasureCoding() *SuperBlockExtra_ErasureCoding {
if m != nil {
@@ -461,7 +518,7 @@ func (m *SuperBlockExtra_ErasureCoding) Reset() { *m = SuperBlockExtra_E
func (m *SuperBlockExtra_ErasureCoding) String() string { return proto.CompactTextString(m) }
func (*SuperBlockExtra_ErasureCoding) ProtoMessage() {}
func (*SuperBlockExtra_ErasureCoding) Descriptor() ([]byte, []int) {
- return fileDescriptor0, []int{6, 0}
+ return fileDescriptor0, []int{7, 0}
}
func (m *SuperBlockExtra_ErasureCoding) GetData() uint32 {
@@ -485,16 +542,16 @@ func (m *SuperBlockExtra_ErasureCoding) GetVolumeIds() []uint32 {
return nil
}
-type ClientListenRequest struct {
+type KeepConnectedRequest struct {
Name string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
}
-func (m *ClientListenRequest) Reset() { *m = ClientListenRequest{} }
-func (m *ClientListenRequest) String() string { return proto.CompactTextString(m) }
-func (*ClientListenRequest) ProtoMessage() {}
-func (*ClientListenRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
+func (m *KeepConnectedRequest) Reset() { *m = KeepConnectedRequest{} }
+func (m *KeepConnectedRequest) String() string { return proto.CompactTextString(m) }
+func (*KeepConnectedRequest) ProtoMessage() {}
+func (*KeepConnectedRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
-func (m *ClientListenRequest) GetName() string {
+func (m *KeepConnectedRequest) GetName() string {
if m != nil {
return m.Name
}
@@ -506,12 +563,13 @@ type VolumeLocation struct {
PublicUrl string `protobuf:"bytes,2,opt,name=public_url,json=publicUrl" json:"public_url,omitempty"`
NewVids []uint32 `protobuf:"varint,3,rep,packed,name=new_vids,json=newVids" json:"new_vids,omitempty"`
DeletedVids []uint32 `protobuf:"varint,4,rep,packed,name=deleted_vids,json=deletedVids" json:"deleted_vids,omitempty"`
+ Leader string `protobuf:"bytes,5,opt,name=leader" json:"leader,omitempty"`
}
func (m *VolumeLocation) Reset() { *m = VolumeLocation{} }
func (m *VolumeLocation) String() string { return proto.CompactTextString(m) }
func (*VolumeLocation) ProtoMessage() {}
-func (*VolumeLocation) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
+func (*VolumeLocation) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
func (m *VolumeLocation) GetUrl() string {
if m != nil {
@@ -541,6 +599,13 @@ func (m *VolumeLocation) GetDeletedVids() []uint32 {
return nil
}
+func (m *VolumeLocation) GetLeader() string {
+ if m != nil {
+ return m.Leader
+ }
+ return ""
+}
+
type LookupVolumeRequest struct {
VolumeIds []string `protobuf:"bytes,1,rep,name=volume_ids,json=volumeIds" json:"volume_ids,omitempty"`
Collection string `protobuf:"bytes,2,opt,name=collection" json:"collection,omitempty"`
@@ -549,7 +614,7 @@ type LookupVolumeRequest struct {
func (m *LookupVolumeRequest) Reset() { *m = LookupVolumeRequest{} }
func (m *LookupVolumeRequest) String() string { return proto.CompactTextString(m) }
func (*LookupVolumeRequest) ProtoMessage() {}
-func (*LookupVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
+func (*LookupVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
func (m *LookupVolumeRequest) GetVolumeIds() []string {
if m != nil {
@@ -572,7 +637,7 @@ type LookupVolumeResponse struct {
func (m *LookupVolumeResponse) Reset() { *m = LookupVolumeResponse{} }
func (m *LookupVolumeResponse) String() string { return proto.CompactTextString(m) }
func (*LookupVolumeResponse) ProtoMessage() {}
-func (*LookupVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
+func (*LookupVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
func (m *LookupVolumeResponse) GetVolumeIdLocations() []*LookupVolumeResponse_VolumeIdLocation {
if m != nil {
@@ -591,7 +656,7 @@ func (m *LookupVolumeResponse_VolumeIdLocation) Reset() { *m = LookupVol
func (m *LookupVolumeResponse_VolumeIdLocation) String() string { return proto.CompactTextString(m) }
func (*LookupVolumeResponse_VolumeIdLocation) ProtoMessage() {}
func (*LookupVolumeResponse_VolumeIdLocation) Descriptor() ([]byte, []int) {
- return fileDescriptor0, []int{10, 0}
+ return fileDescriptor0, []int{11, 0}
}
func (m *LookupVolumeResponse_VolumeIdLocation) GetVolumeId() string {
@@ -623,7 +688,7 @@ type Location struct {
func (m *Location) Reset() { *m = Location{} }
func (m *Location) String() string { return proto.CompactTextString(m) }
func (*Location) ProtoMessage() {}
-func (*Location) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
+func (*Location) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} }
func (m *Location) GetUrl() string {
if m != nil {
@@ -640,19 +705,21 @@ func (m *Location) GetPublicUrl() string {
}
type AssignRequest struct {
- Count uint64 `protobuf:"varint,1,opt,name=count" json:"count,omitempty"`
- Replication string `protobuf:"bytes,2,opt,name=replication" json:"replication,omitempty"`
- Collection string `protobuf:"bytes,3,opt,name=collection" json:"collection,omitempty"`
- Ttl string `protobuf:"bytes,4,opt,name=ttl" json:"ttl,omitempty"`
- DataCenter string `protobuf:"bytes,5,opt,name=data_center,json=dataCenter" json:"data_center,omitempty"`
- Rack string `protobuf:"bytes,6,opt,name=rack" json:"rack,omitempty"`
- DataNode string `protobuf:"bytes,7,opt,name=data_node,json=dataNode" json:"data_node,omitempty"`
+ Count uint64 `protobuf:"varint,1,opt,name=count" json:"count,omitempty"`
+ Replication string `protobuf:"bytes,2,opt,name=replication" json:"replication,omitempty"`
+ Collection string `protobuf:"bytes,3,opt,name=collection" json:"collection,omitempty"`
+ Ttl string `protobuf:"bytes,4,opt,name=ttl" json:"ttl,omitempty"`
+ DataCenter string `protobuf:"bytes,5,opt,name=data_center,json=dataCenter" json:"data_center,omitempty"`
+ Rack string `protobuf:"bytes,6,opt,name=rack" json:"rack,omitempty"`
+ DataNode string `protobuf:"bytes,7,opt,name=data_node,json=dataNode" json:"data_node,omitempty"`
+ MemoryMapMaxSizeMb uint32 `protobuf:"varint,8,opt,name=memory_map_max_size_mb,json=memoryMapMaxSizeMb" json:"memory_map_max_size_mb,omitempty"`
+ WritableVolumeCount uint32 `protobuf:"varint,9,opt,name=Writable_volume_count,json=WritableVolumeCount" json:"Writable_volume_count,omitempty"`
}
func (m *AssignRequest) Reset() { *m = AssignRequest{} }
func (m *AssignRequest) String() string { return proto.CompactTextString(m) }
func (*AssignRequest) ProtoMessage() {}
-func (*AssignRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} }
+func (*AssignRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
func (m *AssignRequest) GetCount() uint64 {
if m != nil {
@@ -703,6 +770,20 @@ func (m *AssignRequest) GetDataNode() string {
return ""
}
+func (m *AssignRequest) GetMemoryMapMaxSizeMb() uint32 {
+ if m != nil {
+ return m.MemoryMapMaxSizeMb
+ }
+ return 0
+}
+
+func (m *AssignRequest) GetWritableVolumeCount() uint32 {
+ if m != nil {
+ return m.WritableVolumeCount
+ }
+ return 0
+}
+
type AssignResponse struct {
Fid string `protobuf:"bytes,1,opt,name=fid" json:"fid,omitempty"`
Url string `protobuf:"bytes,2,opt,name=url" json:"url,omitempty"`
@@ -715,7 +796,7 @@ type AssignResponse struct {
func (m *AssignResponse) Reset() { *m = AssignResponse{} }
func (m *AssignResponse) String() string { return proto.CompactTextString(m) }
func (*AssignResponse) ProtoMessage() {}
-func (*AssignResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
+func (*AssignResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} }
func (m *AssignResponse) GetFid() string {
if m != nil {
@@ -768,7 +849,7 @@ type StatisticsRequest struct {
func (m *StatisticsRequest) Reset() { *m = StatisticsRequest{} }
func (m *StatisticsRequest) String() string { return proto.CompactTextString(m) }
func (*StatisticsRequest) ProtoMessage() {}
-func (*StatisticsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} }
+func (*StatisticsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} }
func (m *StatisticsRequest) GetReplication() string {
if m != nil {
@@ -803,7 +884,7 @@ type StatisticsResponse struct {
func (m *StatisticsResponse) Reset() { *m = StatisticsResponse{} }
func (m *StatisticsResponse) String() string { return proto.CompactTextString(m) }
func (*StatisticsResponse) ProtoMessage() {}
-func (*StatisticsResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} }
+func (*StatisticsResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} }
func (m *StatisticsResponse) GetReplication() string {
if m != nil {
@@ -855,7 +936,7 @@ type StorageType struct {
func (m *StorageType) Reset() { *m = StorageType{} }
func (m *StorageType) String() string { return proto.CompactTextString(m) }
func (*StorageType) ProtoMessage() {}
-func (*StorageType) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} }
+func (*StorageType) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} }
func (m *StorageType) GetReplication() string {
if m != nil {
@@ -878,7 +959,7 @@ type Collection struct {
func (m *Collection) Reset() { *m = Collection{} }
func (m *Collection) String() string { return proto.CompactTextString(m) }
func (*Collection) ProtoMessage() {}
-func (*Collection) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} }
+func (*Collection) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} }
func (m *Collection) GetName() string {
if m != nil {
@@ -895,7 +976,7 @@ type CollectionListRequest struct {
func (m *CollectionListRequest) Reset() { *m = CollectionListRequest{} }
func (m *CollectionListRequest) String() string { return proto.CompactTextString(m) }
func (*CollectionListRequest) ProtoMessage() {}
-func (*CollectionListRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} }
+func (*CollectionListRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} }
func (m *CollectionListRequest) GetIncludeNormalVolumes() bool {
if m != nil {
@@ -918,7 +999,7 @@ type CollectionListResponse struct {
func (m *CollectionListResponse) Reset() { *m = CollectionListResponse{} }
func (m *CollectionListResponse) String() string { return proto.CompactTextString(m) }
func (*CollectionListResponse) ProtoMessage() {}
-func (*CollectionListResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} }
+func (*CollectionListResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} }
func (m *CollectionListResponse) GetCollections() []*Collection {
if m != nil {
@@ -934,7 +1015,7 @@ type CollectionDeleteRequest struct {
func (m *CollectionDeleteRequest) Reset() { *m = CollectionDeleteRequest{} }
func (m *CollectionDeleteRequest) String() string { return proto.CompactTextString(m) }
func (*CollectionDeleteRequest) ProtoMessage() {}
-func (*CollectionDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} }
+func (*CollectionDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} }
func (m *CollectionDeleteRequest) GetName() string {
if m != nil {
@@ -949,7 +1030,7 @@ type CollectionDeleteResponse struct {
func (m *CollectionDeleteResponse) Reset() { *m = CollectionDeleteResponse{} }
func (m *CollectionDeleteResponse) String() string { return proto.CompactTextString(m) }
func (*CollectionDeleteResponse) ProtoMessage() {}
-func (*CollectionDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} }
+func (*CollectionDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} }
//
// volume related
@@ -962,12 +1043,13 @@ type DataNodeInfo struct {
ActiveVolumeCount uint64 `protobuf:"varint,5,opt,name=active_volume_count,json=activeVolumeCount" json:"active_volume_count,omitempty"`
VolumeInfos []*VolumeInformationMessage `protobuf:"bytes,6,rep,name=volume_infos,json=volumeInfos" json:"volume_infos,omitempty"`
EcShardInfos []*VolumeEcShardInformationMessage `protobuf:"bytes,7,rep,name=ec_shard_infos,json=ecShardInfos" json:"ec_shard_infos,omitempty"`
+ RemoteVolumeCount uint64 `protobuf:"varint,8,opt,name=remote_volume_count,json=remoteVolumeCount" json:"remote_volume_count,omitempty"`
}
func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} }
func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) }
func (*DataNodeInfo) ProtoMessage() {}
-func (*DataNodeInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} }
+func (*DataNodeInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} }
func (m *DataNodeInfo) GetId() string {
if m != nil {
@@ -1018,6 +1100,13 @@ func (m *DataNodeInfo) GetEcShardInfos() []*VolumeEcShardInformationMessage {
return nil
}
+func (m *DataNodeInfo) GetRemoteVolumeCount() uint64 {
+ if m != nil {
+ return m.RemoteVolumeCount
+ }
+ return 0
+}
+
type RackInfo struct {
Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
VolumeCount uint64 `protobuf:"varint,2,opt,name=volume_count,json=volumeCount" json:"volume_count,omitempty"`
@@ -1025,12 +1114,13 @@ type RackInfo struct {
FreeVolumeCount uint64 `protobuf:"varint,4,opt,name=free_volume_count,json=freeVolumeCount" json:"free_volume_count,omitempty"`
ActiveVolumeCount uint64 `protobuf:"varint,5,opt,name=active_volume_count,json=activeVolumeCount" json:"active_volume_count,omitempty"`
DataNodeInfos []*DataNodeInfo `protobuf:"bytes,6,rep,name=data_node_infos,json=dataNodeInfos" json:"data_node_infos,omitempty"`
+ RemoteVolumeCount uint64 `protobuf:"varint,7,opt,name=remote_volume_count,json=remoteVolumeCount" json:"remote_volume_count,omitempty"`
}
func (m *RackInfo) Reset() { *m = RackInfo{} }
func (m *RackInfo) String() string { return proto.CompactTextString(m) }
func (*RackInfo) ProtoMessage() {}
-func (*RackInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} }
+func (*RackInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} }
func (m *RackInfo) GetId() string {
if m != nil {
@@ -1074,6 +1164,13 @@ func (m *RackInfo) GetDataNodeInfos() []*DataNodeInfo {
return nil
}
+func (m *RackInfo) GetRemoteVolumeCount() uint64 {
+ if m != nil {
+ return m.RemoteVolumeCount
+ }
+ return 0
+}
+
type DataCenterInfo struct {
Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
VolumeCount uint64 `protobuf:"varint,2,opt,name=volume_count,json=volumeCount" json:"volume_count,omitempty"`
@@ -1081,12 +1178,13 @@ type DataCenterInfo struct {
FreeVolumeCount uint64 `protobuf:"varint,4,opt,name=free_volume_count,json=freeVolumeCount" json:"free_volume_count,omitempty"`
ActiveVolumeCount uint64 `protobuf:"varint,5,opt,name=active_volume_count,json=activeVolumeCount" json:"active_volume_count,omitempty"`
RackInfos []*RackInfo `protobuf:"bytes,6,rep,name=rack_infos,json=rackInfos" json:"rack_infos,omitempty"`
+ RemoteVolumeCount uint64 `protobuf:"varint,7,opt,name=remote_volume_count,json=remoteVolumeCount" json:"remote_volume_count,omitempty"`
}
func (m *DataCenterInfo) Reset() { *m = DataCenterInfo{} }
func (m *DataCenterInfo) String() string { return proto.CompactTextString(m) }
func (*DataCenterInfo) ProtoMessage() {}
-func (*DataCenterInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} }
+func (*DataCenterInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} }
func (m *DataCenterInfo) GetId() string {
if m != nil {
@@ -1130,6 +1228,13 @@ func (m *DataCenterInfo) GetRackInfos() []*RackInfo {
return nil
}
+func (m *DataCenterInfo) GetRemoteVolumeCount() uint64 {
+ if m != nil {
+ return m.RemoteVolumeCount
+ }
+ return 0
+}
+
type TopologyInfo struct {
Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
VolumeCount uint64 `protobuf:"varint,2,opt,name=volume_count,json=volumeCount" json:"volume_count,omitempty"`
@@ -1137,12 +1242,13 @@ type TopologyInfo struct {
FreeVolumeCount uint64 `protobuf:"varint,4,opt,name=free_volume_count,json=freeVolumeCount" json:"free_volume_count,omitempty"`
ActiveVolumeCount uint64 `protobuf:"varint,5,opt,name=active_volume_count,json=activeVolumeCount" json:"active_volume_count,omitempty"`
DataCenterInfos []*DataCenterInfo `protobuf:"bytes,6,rep,name=data_center_infos,json=dataCenterInfos" json:"data_center_infos,omitempty"`
+ RemoteVolumeCount uint64 `protobuf:"varint,7,opt,name=remote_volume_count,json=remoteVolumeCount" json:"remote_volume_count,omitempty"`
}
func (m *TopologyInfo) Reset() { *m = TopologyInfo{} }
func (m *TopologyInfo) String() string { return proto.CompactTextString(m) }
func (*TopologyInfo) ProtoMessage() {}
-func (*TopologyInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} }
+func (*TopologyInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{26} }
func (m *TopologyInfo) GetId() string {
if m != nil {
@@ -1186,13 +1292,20 @@ func (m *TopologyInfo) GetDataCenterInfos() []*DataCenterInfo {
return nil
}
+func (m *TopologyInfo) GetRemoteVolumeCount() uint64 {
+ if m != nil {
+ return m.RemoteVolumeCount
+ }
+ return 0
+}
+
type VolumeListRequest struct {
}
func (m *VolumeListRequest) Reset() { *m = VolumeListRequest{} }
func (m *VolumeListRequest) String() string { return proto.CompactTextString(m) }
func (*VolumeListRequest) ProtoMessage() {}
-func (*VolumeListRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{26} }
+func (*VolumeListRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27} }
type VolumeListResponse struct {
TopologyInfo *TopologyInfo `protobuf:"bytes,1,opt,name=topology_info,json=topologyInfo" json:"topology_info,omitempty"`
@@ -1202,7 +1315,7 @@ type VolumeListResponse struct {
func (m *VolumeListResponse) Reset() { *m = VolumeListResponse{} }
func (m *VolumeListResponse) String() string { return proto.CompactTextString(m) }
func (*VolumeListResponse) ProtoMessage() {}
-func (*VolumeListResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27} }
+func (*VolumeListResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{28} }
func (m *VolumeListResponse) GetTopologyInfo() *TopologyInfo {
if m != nil {
@@ -1225,7 +1338,7 @@ type LookupEcVolumeRequest struct {
func (m *LookupEcVolumeRequest) Reset() { *m = LookupEcVolumeRequest{} }
func (m *LookupEcVolumeRequest) String() string { return proto.CompactTextString(m) }
func (*LookupEcVolumeRequest) ProtoMessage() {}
-func (*LookupEcVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{28} }
+func (*LookupEcVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{29} }
func (m *LookupEcVolumeRequest) GetVolumeId() uint32 {
if m != nil {
@@ -1242,7 +1355,7 @@ type LookupEcVolumeResponse struct {
func (m *LookupEcVolumeResponse) Reset() { *m = LookupEcVolumeResponse{} }
func (m *LookupEcVolumeResponse) String() string { return proto.CompactTextString(m) }
func (*LookupEcVolumeResponse) ProtoMessage() {}
-func (*LookupEcVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{29} }
+func (*LookupEcVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{30} }
func (m *LookupEcVolumeResponse) GetVolumeId() uint32 {
if m != nil {
@@ -1269,7 +1382,7 @@ func (m *LookupEcVolumeResponse_EcShardIdLocation) Reset() {
func (m *LookupEcVolumeResponse_EcShardIdLocation) String() string { return proto.CompactTextString(m) }
func (*LookupEcVolumeResponse_EcShardIdLocation) ProtoMessage() {}
func (*LookupEcVolumeResponse_EcShardIdLocation) Descriptor() ([]byte, []int) {
- return fileDescriptor0, []int{29, 0}
+ return fileDescriptor0, []int{30, 0}
}
func (m *LookupEcVolumeResponse_EcShardIdLocation) GetShardId() uint32 {
@@ -1292,7 +1405,7 @@ type GetMasterConfigurationRequest struct {
func (m *GetMasterConfigurationRequest) Reset() { *m = GetMasterConfigurationRequest{} }
func (m *GetMasterConfigurationRequest) String() string { return proto.CompactTextString(m) }
func (*GetMasterConfigurationRequest) ProtoMessage() {}
-func (*GetMasterConfigurationRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{30} }
+func (*GetMasterConfigurationRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{31} }
type GetMasterConfigurationResponse struct {
MetricsAddress string `protobuf:"bytes,1,opt,name=metrics_address,json=metricsAddress" json:"metrics_address,omitempty"`
@@ -1302,7 +1415,7 @@ type GetMasterConfigurationResponse struct {
func (m *GetMasterConfigurationResponse) Reset() { *m = GetMasterConfigurationResponse{} }
func (m *GetMasterConfigurationResponse) String() string { return proto.CompactTextString(m) }
func (*GetMasterConfigurationResponse) ProtoMessage() {}
-func (*GetMasterConfigurationResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{31} }
+func (*GetMasterConfigurationResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{32} }
func (m *GetMasterConfigurationResponse) GetMetricsAddress() string {
if m != nil {
@@ -1324,10 +1437,11 @@ func init() {
proto.RegisterType((*VolumeInformationMessage)(nil), "master_pb.VolumeInformationMessage")
proto.RegisterType((*VolumeShortInformationMessage)(nil), "master_pb.VolumeShortInformationMessage")
proto.RegisterType((*VolumeEcShardInformationMessage)(nil), "master_pb.VolumeEcShardInformationMessage")
+ proto.RegisterType((*StorageBackend)(nil), "master_pb.StorageBackend")
proto.RegisterType((*Empty)(nil), "master_pb.Empty")
proto.RegisterType((*SuperBlockExtra)(nil), "master_pb.SuperBlockExtra")
proto.RegisterType((*SuperBlockExtra_ErasureCoding)(nil), "master_pb.SuperBlockExtra.ErasureCoding")
- proto.RegisterType((*ClientListenRequest)(nil), "master_pb.ClientListenRequest")
+ proto.RegisterType((*KeepConnectedRequest)(nil), "master_pb.KeepConnectedRequest")
proto.RegisterType((*VolumeLocation)(nil), "master_pb.VolumeLocation")
proto.RegisterType((*LookupVolumeRequest)(nil), "master_pb.LookupVolumeRequest")
proto.RegisterType((*LookupVolumeResponse)(nil), "master_pb.LookupVolumeResponse")
@@ -1428,7 +1542,7 @@ func (c *seaweedClient) KeepConnected(ctx context.Context, opts ...grpc.CallOpti
}
type Seaweed_KeepConnectedClient interface {
- Send(*ClientListenRequest) error
+ Send(*KeepConnectedRequest) error
Recv() (*VolumeLocation, error)
grpc.ClientStream
}
@@ -1437,7 +1551,7 @@ type seaweedKeepConnectedClient struct {
grpc.ClientStream
}
-func (x *seaweedKeepConnectedClient) Send(m *ClientListenRequest) error {
+func (x *seaweedKeepConnectedClient) Send(m *KeepConnectedRequest) error {
return x.ClientStream.SendMsg(m)
}
@@ -1572,7 +1686,7 @@ func _Seaweed_KeepConnected_Handler(srv interface{}, stream grpc.ServerStream) e
type Seaweed_KeepConnectedServer interface {
Send(*VolumeLocation) error
- Recv() (*ClientListenRequest, error)
+ Recv() (*KeepConnectedRequest, error)
grpc.ServerStream
}
@@ -1584,8 +1698,8 @@ func (x *seaweedKeepConnectedServer) Send(m *VolumeLocation) error {
return x.ServerStream.SendMsg(m)
}
-func (x *seaweedKeepConnectedServer) Recv() (*ClientListenRequest, error) {
- m := new(ClientListenRequest)
+func (x *seaweedKeepConnectedServer) Recv() (*KeepConnectedRequest, error) {
+ m := new(KeepConnectedRequest)
if err := x.ServerStream.RecvMsg(m); err != nil {
return nil, err
}
@@ -1793,123 +1907,137 @@ var _Seaweed_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("master.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{
- // 1888 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xd4, 0x58, 0xcd, 0x6f, 0x1c, 0x49,
- 0x15, 0x4f, 0xcf, 0x8c, 0xc7, 0x33, 0x6f, 0x3e, 0x3c, 0x53, 0x76, 0xbc, 0x93, 0x59, 0x92, 0x4c,
- 0x7a, 0x91, 0xd6, 0x09, 0x8b, 0x59, 0xb2, 0x2b, 0x81, 0x04, 0x68, 0x95, 0x38, 0xde, 0xc5, 0xca,
- 0xc7, 0x3a, 0x3d, 0x21, 0x48, 0x48, 0xa8, 0xa9, 0xe9, 0x2e, 0xdb, 0x25, 0xf7, 0x74, 0x37, 0x5d,
- 0x35, 0x8e, 0x67, 0x39, 0x70, 0x80, 0x1b, 0x12, 0x17, 0xce, 0xdc, 0xf9, 0x1b, 0x38, 0x70, 0xe1,
- 0xc8, 0x9d, 0x3b, 0xff, 0x02, 0x57, 0x84, 0xb4, 0xaa, 0xaf, 0xee, 0xea, 0x99, 0xb1, 0x1d, 0xaf,
- 0xb4, 0x87, 0xdc, 0xaa, 0xdf, 0x7b, 0xf5, 0xde, 0xab, 0xdf, 0xab, 0xf7, 0x51, 0x0d, 0xed, 0x29,
- 0x66, 0x9c, 0x64, 0xbb, 0x69, 0x96, 0xf0, 0x04, 0x35, 0xd5, 0x97, 0x9f, 0x4e, 0xdc, 0x3f, 0xd5,
- 0xa1, 0xf9, 0x73, 0x82, 0x33, 0x3e, 0x21, 0x98, 0xa3, 0x2e, 0x54, 0x68, 0x3a, 0x70, 0x46, 0xce,
- 0x4e, 0xd3, 0xab, 0xd0, 0x14, 0x21, 0xa8, 0xa5, 0x49, 0xc6, 0x07, 0x95, 0x91, 0xb3, 0xd3, 0xf1,
- 0xe4, 0x1a, 0xdd, 0x06, 0x48, 0x67, 0x93, 0x88, 0x06, 0xfe, 0x2c, 0x8b, 0x06, 0x55, 0x29, 0xdb,
- 0x54, 0x94, 0x5f, 0x64, 0x11, 0xda, 0x81, 0xde, 0x14, 0x9f, 0xfb, 0x67, 0x49, 0x34, 0x9b, 0x12,
- 0x3f, 0x48, 0x66, 0x31, 0x1f, 0xd4, 0xe4, 0xf6, 0xee, 0x14, 0x9f, 0xbf, 0x96, 0xe4, 0x3d, 0x41,
- 0x45, 0x23, 0xe1, 0xd5, 0xb9, 0x7f, 0x44, 0x23, 0xe2, 0x9f, 0x92, 0xf9, 0x60, 0x6d, 0xe4, 0xec,
- 0xd4, 0x3c, 0x98, 0xe2, 0xf3, 0xcf, 0x69, 0x44, 0x9e, 0x92, 0x39, 0xba, 0x0b, 0xad, 0x10, 0x73,
- 0xec, 0x07, 0x24, 0xe6, 0x24, 0x1b, 0xd4, 0xa5, 0x2d, 0x10, 0xa4, 0x3d, 0x49, 0x11, 0xfe, 0x65,
- 0x38, 0x38, 0x1d, 0xac, 0x4b, 0x8e, 0x5c, 0x0b, 0xff, 0x70, 0x38, 0xa5, 0xb1, 0x2f, 0x3d, 0x6f,
- 0x48, 0xd3, 0x4d, 0x49, 0x39, 0x14, 0xee, 0xff, 0x0c, 0xd6, 0x95, 0x6f, 0x6c, 0xd0, 0x1c, 0x55,
- 0x77, 0x5a, 0x0f, 0x3f, 0xd8, 0xcd, 0xd1, 0xd8, 0x55, 0xee, 0x1d, 0xc4, 0x47, 0x49, 0x36, 0xc5,
- 0x9c, 0x26, 0xf1, 0x73, 0xc2, 0x18, 0x3e, 0x26, 0x9e, 0xd9, 0x83, 0x0e, 0xa0, 0x15, 0x93, 0x37,
- 0xbe, 0x51, 0x01, 0x52, 0xc5, 0xce, 0x92, 0x8a, 0xf1, 0x49, 0x92, 0xf1, 0x15, 0x7a, 0x20, 0x26,
- 0x6f, 0x5e, 0x6b, 0x55, 0x2f, 0x61, 0x23, 0x24, 0x11, 0xe1, 0x24, 0xcc, 0xd5, 0xb5, 0xae, 0xa9,
- 0xae, 0xab, 0x15, 0x18, 0x95, 0xdf, 0x85, 0xee, 0x09, 0x66, 0x7e, 0x9c, 0xe4, 0x1a, 0xdb, 0x23,
- 0x67, 0xa7, 0xe1, 0xb5, 0x4f, 0x30, 0x7b, 0x91, 0x18, 0xa9, 0x2f, 0xa0, 0x49, 0x02, 0x9f, 0x9d,
- 0xe0, 0x2c, 0x64, 0x83, 0x9e, 0x34, 0xf9, 0x60, 0xc9, 0xe4, 0x7e, 0x30, 0x16, 0x02, 0x2b, 0x8c,
- 0x36, 0x88, 0x62, 0x31, 0xf4, 0x02, 0x3a, 0x02, 0x8c, 0x42, 0x59, 0xff, 0xda, 0xca, 0x04, 0x9a,
- 0xfb, 0x46, 0xdf, 0x6b, 0xe8, 0x1b, 0x44, 0x0a, 0x9d, 0xe8, 0xda, 0x3a, 0x0d, 0xac, 0xb9, 0xde,
- 0x0f, 0xa1, 0xa7, 0x61, 0x29, 0xd4, 0x6e, 0x4a, 0x60, 0x3a, 0x12, 0x18, 0x23, 0xe8, 0xfe, 0xdd,
- 0x81, 0x7e, 0x9e, 0x0d, 0x1e, 0x61, 0x69, 0x12, 0x33, 0x82, 0x1e, 0x40, 0x5f, 0x5f, 0x67, 0x46,
- 0xbf, 0x22, 0x7e, 0x44, 0xa7, 0x94, 0xcb, 0x24, 0xa9, 0x79, 0x1b, 0x8a, 0x31, 0xa6, 0x5f, 0x91,
- 0x67, 0x82, 0x8c, 0xb6, 0xa1, 0x1e, 0x11, 0x1c, 0x92, 0x4c, 0xe6, 0x4c, 0xd3, 0xd3, 0x5f, 0xe8,
- 0x43, 0xd8, 0x98, 0x12, 0x9e, 0xd1, 0x80, 0xf9, 0x38, 0x0c, 0x33, 0xc2, 0x98, 0x4e, 0x9d, 0xae,
- 0x26, 0x3f, 0x52, 0x54, 0xf4, 0x63, 0x18, 0x18, 0x41, 0x2a, 0xee, 0xf8, 0x19, 0x8e, 0x7c, 0x46,
- 0x82, 0x24, 0x0e, 0x99, 0xce, 0xa3, 0x6d, 0xcd, 0x3f, 0xd0, 0xec, 0xb1, 0xe2, 0xba, 0x7f, 0xad,
- 0xc2, 0xe0, 0xa2, 0x0b, 0x2c, 0x33, 0x3b, 0x94, 0x4e, 0x77, 0xbc, 0x0a, 0x0d, 0x45, 0xe6, 0x88,
- 0xc3, 0x48, 0x2f, 0x6b, 0x9e, 0x5c, 0xa3, 0x3b, 0x00, 0x41, 0x12, 0x45, 0x24, 0x10, 0x1b, 0xb5,
- 0x7b, 0x16, 0x45, 0x64, 0x96, 0x4c, 0xd6, 0x22, 0xa9, 0x6b, 0x5e, 0x53, 0x50, 0x54, 0x3e, 0xdf,
- 0x83, 0xb6, 0x02, 0x5e, 0x0b, 0xa8, 0x7c, 0x6e, 0x29, 0x9a, 0x12, 0xf9, 0x08, 0x90, 0x09, 0xf0,
- 0x64, 0x9e, 0x0b, 0xd6, 0xa5, 0x60, 0x4f, 0x73, 0x1e, 0xcf, 0x8d, 0xf4, 0xfb, 0xd0, 0xcc, 0x08,
- 0x0e, 0xfd, 0x24, 0x8e, 0xe6, 0x32, 0xc5, 0x1b, 0x5e, 0x43, 0x10, 0xbe, 0x8c, 0xa3, 0x39, 0xfa,
- 0x1e, 0xf4, 0x33, 0x92, 0x46, 0x34, 0xc0, 0x7e, 0x1a, 0xe1, 0x80, 0x4c, 0x49, 0x6c, 0xb2, 0xbd,
- 0xa7, 0x19, 0x87, 0x86, 0x8e, 0x06, 0xb0, 0x7e, 0x46, 0x32, 0x26, 0x8e, 0xd5, 0x94, 0x22, 0xe6,
- 0x13, 0xf5, 0xa0, 0xca, 0x79, 0x34, 0x00, 0x49, 0x15, 0x4b, 0x74, 0x1f, 0x7a, 0x41, 0x32, 0x4d,
- 0x71, 0xc0, 0xfd, 0x8c, 0x9c, 0x51, 0xb9, 0xa9, 0x25, 0xd9, 0x1b, 0x9a, 0xee, 0x69, 0xb2, 0x38,
- 0xce, 0x34, 0x09, 0xe9, 0x11, 0x25, 0xa1, 0x8f, 0xb9, 0x0e, 0x93, 0x4c, 0xb9, 0xaa, 0xd7, 0x33,
- 0x9c, 0x47, 0x5c, 0x05, 0xc8, 0xfd, 0x9b, 0x03, 0xb7, 0x2f, 0x4d, 0xe7, 0xa5, 0x20, 0x5d, 0x15,
- 0x90, 0x6f, 0x0b, 0x03, 0x77, 0x06, 0x77, 0xaf, 0x48, 0xb2, 0x2b, 0x7c, 0xad, 0x2c, 0xf9, 0xea,
- 0x42, 0x87, 0x04, 0x3e, 0x8d, 0x43, 0x72, 0xee, 0x4f, 0x28, 0x57, 0xd7, 0xbf, 0xe3, 0xb5, 0x48,
- 0x70, 0x20, 0x68, 0x8f, 0x29, 0x67, 0xee, 0x3a, 0xac, 0xed, 0x4f, 0x53, 0x3e, 0x77, 0xff, 0xe1,
- 0xc0, 0xc6, 0x78, 0x96, 0x92, 0xec, 0x71, 0x94, 0x04, 0xa7, 0xfb, 0xe7, 0x3c, 0xc3, 0xe8, 0x4b,
- 0xe8, 0x92, 0x0c, 0xb3, 0x59, 0x26, 0xae, 0x4d, 0x48, 0xe3, 0x63, 0x69, 0xbc, 0x5c, 0x2d, 0x17,
- 0xf6, 0xec, 0xee, 0xab, 0x0d, 0x7b, 0x52, 0xde, 0xeb, 0x10, 0xfb, 0x73, 0xf8, 0x2b, 0xe8, 0x94,
- 0xf8, 0x22, 0x27, 0x44, 0x6f, 0xd1, 0x87, 0x92, 0x6b, 0x91, 0xcf, 0x29, 0xce, 0x28, 0x9f, 0xeb,
- 0x1e, 0xa8, 0xbf, 0x44, 0x2e, 0xe8, 0x9a, 0x40, 0x43, 0x71, 0x96, 0xaa, 0xe8, 0x32, 0x8a, 0x72,
- 0x10, 0x32, 0xf7, 0x3e, 0x6c, 0xee, 0x45, 0x94, 0xc4, 0xfc, 0x19, 0x65, 0x9c, 0xc4, 0x1e, 0xf9,
- 0xed, 0x8c, 0x30, 0x2e, 0x2c, 0xc4, 0x78, 0x4a, 0x74, 0x87, 0x95, 0x6b, 0xf7, 0xf7, 0xd0, 0x55,
- 0x58, 0x3f, 0x4b, 0x02, 0x89, 0xb0, 0x88, 0x87, 0x68, 0xad, 0x4a, 0x48, 0x2c, 0x17, 0x7a, 0x6e,
- 0x65, 0xb1, 0xe7, 0xde, 0x82, 0x86, 0x6c, 0x4a, 0x85, 0x2b, 0xeb, 0xa2, 0xcf, 0xd0, 0x90, 0x15,
- 0x49, 0x19, 0x2a, 0x76, 0x4d, 0xb2, 0x5b, 0xa6, 0x6f, 0xd0, 0x90, 0xb9, 0xaf, 0x60, 0xf3, 0x59,
- 0x92, 0x9c, 0xce, 0x52, 0xe5, 0x86, 0xf1, 0xb5, 0x7c, 0x42, 0x67, 0x54, 0x15, 0x36, 0xf3, 0x13,
- 0x5e, 0x15, 0x6f, 0xf7, 0xbf, 0x0e, 0x6c, 0x95, 0xd5, 0xea, 0x6a, 0xfa, 0x1b, 0xd8, 0xcc, 0xf5,
- 0xfa, 0x91, 0x3e, 0xb3, 0x32, 0xd0, 0x7a, 0xf8, 0xb1, 0x15, 0xcc, 0x55, 0xbb, 0x4d, 0x87, 0x0e,
- 0x0d, 0x58, 0x5e, 0xff, 0x6c, 0x81, 0xc2, 0x86, 0xe7, 0xd0, 0x5b, 0x14, 0x13, 0xb5, 0x24, 0xb7,
- 0xaa, 0x91, 0x6d, 0x98, 0x9d, 0xe8, 0x87, 0xd0, 0x2c, 0x1c, 0xa9, 0x48, 0x47, 0x36, 0x4b, 0x8e,
- 0x68, 0x5b, 0x85, 0x14, 0xda, 0x82, 0x35, 0x92, 0x65, 0x49, 0xa6, 0xb3, 0x52, 0x7d, 0xb8, 0x3f,
- 0x81, 0xc6, 0x37, 0x8e, 0xa2, 0xfb, 0x2f, 0x07, 0x3a, 0x8f, 0x18, 0xa3, 0xc7, 0xf9, 0x75, 0xd9,
- 0x82, 0x35, 0x55, 0x21, 0x55, 0xb3, 0x51, 0x1f, 0x68, 0x04, 0x2d, 0x9d, 0xdc, 0x16, 0xf4, 0x36,
- 0xe9, 0xca, 0xba, 0xa1, 0x13, 0xbe, 0xa6, 0x5c, 0x13, 0x45, 0x6f, 0x61, 0xd2, 0x5a, 0xbb, 0x70,
- 0xd2, 0xaa, 0x5b, 0x93, 0xd6, 0xfb, 0xd0, 0x94, 0x9b, 0xe2, 0x24, 0x24, 0x7a, 0x04, 0x6b, 0x08,
- 0xc2, 0x8b, 0x24, 0x24, 0xee, 0x5f, 0x1c, 0xe8, 0x9a, 0xd3, 0xe8, 0xc8, 0xf7, 0xa0, 0x7a, 0x94,
- 0xa3, 0x2f, 0x96, 0x06, 0xa3, 0xca, 0x45, 0x18, 0x2d, 0x4d, 0x97, 0x39, 0x22, 0x35, 0x1b, 0x91,
- 0x3c, 0x18, 0x6b, 0x56, 0x30, 0x84, 0xcb, 0x78, 0xc6, 0x4f, 0x8c, 0xcb, 0x62, 0xed, 0x1e, 0x43,
- 0x7f, 0xcc, 0x31, 0xa7, 0x8c, 0xd3, 0x80, 0x19, 0x98, 0x17, 0x00, 0x75, 0xae, 0x02, 0xb4, 0x72,
- 0x11, 0xa0, 0xd5, 0x1c, 0x50, 0xf7, 0x9f, 0x0e, 0x20, 0xdb, 0x92, 0x86, 0xe0, 0x5b, 0x30, 0x25,
- 0x20, 0xe3, 0x09, 0x17, 0x63, 0x82, 0x68, 0xe8, 0xba, 0x2d, 0x4b, 0x8a, 0x18, 0x4b, 0x44, 0x94,
- 0x66, 0x8c, 0x84, 0x8a, 0xab, 0x7a, 0x72, 0x43, 0x10, 0x24, 0xb3, 0xdc, 0xd2, 0xeb, 0x0b, 0x2d,
- 0xdd, 0x7d, 0x04, 0xad, 0x31, 0x4f, 0x32, 0x7c, 0x4c, 0x5e, 0xcd, 0xd3, 0xb7, 0xf1, 0x5e, 0x7b,
- 0x57, 0x29, 0x80, 0x18, 0x01, 0xec, 0x15, 0xde, 0xaf, 0x2a, 0x80, 0xbf, 0x83, 0x9b, 0x85, 0x84,
- 0xa8, 0x97, 0x26, 0x2e, 0x9f, 0xc2, 0x36, 0x8d, 0x83, 0x68, 0x16, 0x12, 0x3f, 0x16, 0xed, 0x27,
- 0xca, 0xa7, 0x5a, 0x47, 0x0e, 0x03, 0x5b, 0x9a, 0xfb, 0x42, 0x32, 0xcd, 0x74, 0xfb, 0x11, 0x20,
- 0xb3, 0x8b, 0x04, 0xf9, 0x8e, 0x8a, 0xdc, 0xd1, 0xd3, 0x9c, 0xfd, 0x40, 0x4b, 0xbb, 0x2f, 0x61,
- 0x7b, 0xd1, 0xb8, 0x0e, 0xd5, 0x8f, 0xa0, 0x55, 0xc0, 0x6e, 0xea, 0xd3, 0x4d, 0xab, 0x2c, 0x14,
- 0xfb, 0x3c, 0x5b, 0xd2, 0xfd, 0x3e, 0xbc, 0x57, 0xb0, 0x9e, 0xc8, 0x42, 0x7b, 0x59, 0xfd, 0x1f,
- 0xc2, 0x60, 0x59, 0x5c, 0xf9, 0xe0, 0xfe, 0xa7, 0x02, 0xed, 0x27, 0x3a, 0xa3, 0x44, 0x0f, 0xb6,
- 0xba, 0x6e, 0x53, 0x76, 0xdd, 0x7b, 0xd0, 0x2e, 0xbd, 0xb4, 0xd4, 0x38, 0xd7, 0x3a, 0xb3, 0x9e,
- 0x59, 0xab, 0x1e, 0x64, 0x55, 0x29, 0xb6, 0xf8, 0x20, 0x7b, 0x00, 0xfd, 0xa3, 0x8c, 0x90, 0xe5,
- 0xb7, 0x5b, 0xcd, 0xdb, 0x10, 0x0c, 0x5b, 0x76, 0x17, 0x36, 0x71, 0xc0, 0xe9, 0xd9, 0x82, 0xb4,
- 0xba, 0x5f, 0x7d, 0xc5, 0xb2, 0xe5, 0x3f, 0xcf, 0x1d, 0xa5, 0xf1, 0x51, 0xc2, 0x06, 0xf5, 0xb7,
- 0x7f, 0x7b, 0xe9, 0xd3, 0x08, 0x0e, 0x43, 0x87, 0xd0, 0x35, 0x33, 0xbc, 0xd6, 0xb4, 0x7e, 0xed,
- 0xf7, 0x41, 0x9b, 0x14, 0x2c, 0xe6, 0xfe, 0xb1, 0x02, 0x0d, 0x0f, 0x07, 0xa7, 0xef, 0x36, 0xbe,
- 0x9f, 0xc1, 0x46, 0x5e, 0x8b, 0x4b, 0x10, 0xbf, 0x67, 0x01, 0x63, 0x5f, 0x25, 0xaf, 0x13, 0x5a,
- 0x5f, 0xcc, 0xfd, 0xbf, 0x03, 0xdd, 0x27, 0x79, 0xbd, 0x7f, 0xb7, 0xc1, 0x78, 0x08, 0x20, 0x1a,
- 0x54, 0x09, 0x07, 0xbb, 0xa1, 0x9b, 0x70, 0x7b, 0xcd, 0x4c, 0xaf, 0x98, 0xfb, 0xe7, 0x0a, 0xb4,
- 0x5f, 0x25, 0x69, 0x12, 0x25, 0xc7, 0xf3, 0x77, 0xfb, 0xf4, 0xfb, 0xd0, 0xb7, 0x7a, 0x79, 0x09,
- 0x84, 0x5b, 0x0b, 0x97, 0xa1, 0x08, 0xb6, 0xb7, 0x11, 0x96, 0xbe, 0x99, 0xbb, 0x09, 0x7d, 0x3d,
- 0x97, 0x16, 0x25, 0xd9, 0xfd, 0x83, 0x03, 0xc8, 0xa6, 0xea, 0x5a, 0xf9, 0x53, 0xe8, 0x70, 0x8d,
- 0x9d, 0xb4, 0xa7, 0x47, 0x73, 0xfb, 0xee, 0xd9, 0xd8, 0x7a, 0x6d, 0x6e, 0x23, 0xfd, 0x03, 0xd8,
- 0x5a, 0x7a, 0x5f, 0xfb, 0xd3, 0x89, 0x46, 0xb8, 0xbf, 0xf0, 0xc4, 0x7e, 0x3e, 0x71, 0x3f, 0x85,
- 0x9b, 0x6a, 0x38, 0x34, 0x75, 0xdc, 0xd4, 0xd7, 0xa5, 0x29, 0xaf, 0x53, 0x4c, 0x79, 0xee, 0xff,
- 0x1c, 0xd8, 0x5e, 0xdc, 0xa6, 0xfd, 0xbf, 0x6c, 0x1f, 0xc2, 0x80, 0x74, 0xbd, 0xb1, 0xe7, 0x55,
- 0x35, 0x26, 0x7e, 0xb2, 0x34, 0xaf, 0x2e, 0xea, 0xde, 0x35, 0x75, 0xa8, 0x18, 0x59, 0x7b, 0xac,
- 0x4c, 0x60, 0x43, 0x0c, 0xfd, 0x25, 0x31, 0x31, 0xd5, 0x1b, 0xbb, 0xda, 0xa7, 0x75, 0xbd, 0xf1,
- 0x1b, 0x0c, 0xac, 0xee, 0x5d, 0xb8, 0xfd, 0x05, 0xe1, 0xcf, 0xa5, 0xcc, 0x5e, 0x12, 0x1f, 0xd1,
- 0xe3, 0x59, 0xa6, 0x84, 0x8a, 0xd0, 0xde, 0xb9, 0x48, 0x42, 0xc3, 0xb4, 0xe2, 0x27, 0x86, 0x73,
- 0xed, 0x9f, 0x18, 0x95, 0xcb, 0x7e, 0x62, 0x3c, 0xfc, 0x77, 0x1d, 0xd6, 0xc7, 0x04, 0xbf, 0x21,
- 0x24, 0x44, 0x07, 0xd0, 0x19, 0x93, 0x38, 0x2c, 0x7e, 0x4f, 0x6e, 0x59, 0x67, 0xcc, 0xa9, 0xc3,
- 0xef, 0xac, 0xa2, 0xe6, 0x2d, 0xf4, 0xc6, 0x8e, 0xf3, 0xb1, 0x83, 0x0e, 0xa1, 0xf3, 0x94, 0x90,
- 0x74, 0x2f, 0x89, 0x63, 0x12, 0x70, 0x12, 0xa2, 0x3b, 0x76, 0x23, 0x5f, 0x7e, 0xa9, 0x0d, 0x6f,
- 0x2d, 0xf5, 0x13, 0x03, 0xaa, 0xd6, 0xf8, 0x12, 0xda, 0xf6, 0x03, 0xa5, 0xa4, 0x70, 0xc5, 0x73,
- 0x6a, 0x78, 0xf7, 0x8a, 0x97, 0x8d, 0x7b, 0x03, 0x7d, 0x06, 0x75, 0x35, 0x31, 0xa3, 0x81, 0x25,
- 0x5c, 0x7a, 0x12, 0x94, 0xfc, 0x2a, 0x8f, 0xd7, 0xee, 0x0d, 0xf4, 0x14, 0xa0, 0x98, 0x39, 0x91,
- 0x8d, 0xcb, 0xd2, 0xd0, 0x3b, 0xbc, 0x7d, 0x01, 0x37, 0x57, 0xf6, 0x4b, 0xe8, 0x96, 0x27, 0x23,
- 0x34, 0x5a, 0x39, 0xfc, 0x58, 0xe5, 0x61, 0x78, 0xef, 0x12, 0x89, 0x5c, 0xf1, 0xaf, 0xa1, 0xb7,
- 0x38, 0xf0, 0x20, 0x77, 0xe5, 0xc6, 0xd2, 0xf0, 0x34, 0xfc, 0xe0, 0x52, 0x19, 0x1b, 0x84, 0xa2,
- 0x42, 0x95, 0x40, 0x58, 0x2a, 0x67, 0x25, 0x10, 0x96, 0xcb, 0x9a, 0x02, 0xa1, 0x9c, 0xd6, 0x25,
- 0x10, 0x56, 0x16, 0xa1, 0x12, 0x08, 0xab, 0x6b, 0x82, 0x7b, 0x03, 0x25, 0xb0, 0xbd, 0x3a, 0xd9,
- 0x90, 0xfd, 0x3f, 0xe3, 0xd2, 0x8c, 0x1d, 0xde, 0x7f, 0x0b, 0x49, 0x63, 0x70, 0x52, 0x97, 0xbf,
- 0xfe, 0x3f, 0xf9, 0x3a, 0x00, 0x00, 0xff, 0xff, 0x28, 0x1f, 0x4c, 0x0e, 0x0a, 0x18, 0x00, 0x00,
+ // 2102 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xd4, 0x59, 0x4b, 0x6f, 0x1c, 0xc7,
+ 0x11, 0xd6, 0xec, 0x2e, 0x97, 0xbb, 0xb5, 0xef, 0x26, 0x45, 0xaf, 0xd6, 0x96, 0xb5, 0x1a, 0x07,
+ 0x30, 0xa5, 0x38, 0x8c, 0x43, 0x1b, 0x88, 0x91, 0xc4, 0x30, 0x24, 0x8a, 0x76, 0x08, 0x89, 0xb4,
+ 0x34, 0x54, 0x64, 0x20, 0x40, 0x30, 0xe9, 0x9d, 0x69, 0x92, 0x03, 0xce, 0x2b, 0xd3, 0xbd, 0x14,
+ 0xd7, 0xb9, 0x04, 0xc8, 0x31, 0xa7, 0x20, 0x87, 0xfc, 0x85, 0x5c, 0x72, 0x4a, 0xce, 0xbe, 0xe4,
+ 0x47, 0xe4, 0x7f, 0xe4, 0xea, 0x4b, 0xd0, 0xaf, 0x99, 0x9e, 0x7d, 0x90, 0xa6, 0x01, 0x1f, 0x74,
+ 0x9b, 0xae, 0xaa, 0xae, 0xae, 0xfe, 0xaa, 0xbb, 0xea, 0xeb, 0x5d, 0x68, 0x47, 0x98, 0x32, 0x92,
+ 0xed, 0xa4, 0x59, 0xc2, 0x12, 0xd4, 0x94, 0x23, 0x37, 0x9d, 0xd8, 0x7f, 0xa9, 0x43, 0xf3, 0xd7,
+ 0x04, 0x67, 0x6c, 0x42, 0x30, 0x43, 0x5d, 0xa8, 0x04, 0xe9, 0xd0, 0x1a, 0x5b, 0xdb, 0x4d, 0xa7,
+ 0x12, 0xa4, 0x08, 0x41, 0x2d, 0x4d, 0x32, 0x36, 0xac, 0x8c, 0xad, 0xed, 0x8e, 0x23, 0xbe, 0xd1,
+ 0x5d, 0x80, 0x74, 0x3a, 0x09, 0x03, 0xcf, 0x9d, 0x66, 0xe1, 0xb0, 0x2a, 0x6c, 0x9b, 0x52, 0xf2,
+ 0x9b, 0x2c, 0x44, 0xdb, 0xd0, 0x8f, 0xf0, 0xa5, 0x7b, 0x91, 0x84, 0xd3, 0x88, 0xb8, 0x5e, 0x32,
+ 0x8d, 0xd9, 0xb0, 0x26, 0xa6, 0x77, 0x23, 0x7c, 0xf9, 0x4a, 0x88, 0xf7, 0xb8, 0x14, 0x8d, 0x79,
+ 0x54, 0x97, 0xee, 0x49, 0x10, 0x12, 0xf7, 0x9c, 0xcc, 0x86, 0x6b, 0x63, 0x6b, 0xbb, 0xe6, 0x40,
+ 0x84, 0x2f, 0x3f, 0x0f, 0x42, 0xf2, 0x94, 0xcc, 0xd0, 0x3d, 0x68, 0xf9, 0x98, 0x61, 0xd7, 0x23,
+ 0x31, 0x23, 0xd9, 0xb0, 0x2e, 0xd6, 0x02, 0x2e, 0xda, 0x13, 0x12, 0x1e, 0x5f, 0x86, 0xbd, 0xf3,
+ 0xe1, 0xba, 0xd0, 0x88, 0x6f, 0x1e, 0x1f, 0xf6, 0xa3, 0x20, 0x76, 0x45, 0xe4, 0x0d, 0xb1, 0x74,
+ 0x53, 0x48, 0x9e, 0xf3, 0xf0, 0x3f, 0x85, 0x75, 0x19, 0x1b, 0x1d, 0x36, 0xc7, 0xd5, 0xed, 0xd6,
+ 0xee, 0x7b, 0x3b, 0x39, 0x1a, 0x3b, 0x32, 0xbc, 0x83, 0xf8, 0x24, 0xc9, 0x22, 0xcc, 0x82, 0x24,
+ 0x3e, 0x24, 0x94, 0xe2, 0x53, 0xe2, 0xe8, 0x39, 0xe8, 0x00, 0x5a, 0x31, 0x79, 0xed, 0x6a, 0x17,
+ 0x20, 0x5c, 0x6c, 0x2f, 0xb8, 0x38, 0x3e, 0x4b, 0x32, 0xb6, 0xc4, 0x0f, 0xc4, 0xe4, 0xf5, 0x2b,
+ 0xe5, 0xea, 0x05, 0xf4, 0x7c, 0x12, 0x12, 0x46, 0xfc, 0xdc, 0x5d, 0xeb, 0x86, 0xee, 0xba, 0xca,
+ 0x81, 0x76, 0xf9, 0x23, 0xe8, 0x9e, 0x61, 0xea, 0xc6, 0x49, 0xee, 0xb1, 0x3d, 0xb6, 0xb6, 0x1b,
+ 0x4e, 0xfb, 0x0c, 0xd3, 0xa3, 0x44, 0x5b, 0x7d, 0x01, 0x4d, 0xe2, 0xb9, 0xf4, 0x0c, 0x67, 0x3e,
+ 0x1d, 0xf6, 0xc5, 0x92, 0x0f, 0x17, 0x96, 0xdc, 0xf7, 0x8e, 0xb9, 0xc1, 0x92, 0x45, 0x1b, 0x44,
+ 0xaa, 0x28, 0x3a, 0x82, 0x0e, 0x07, 0xa3, 0x70, 0x36, 0xb8, 0xb1, 0x33, 0x8e, 0xe6, 0xbe, 0xf6,
+ 0xf7, 0x0a, 0x06, 0x1a, 0x91, 0xc2, 0x27, 0xba, 0xb1, 0x4f, 0x0d, 0x6b, 0xee, 0xf7, 0x7d, 0xe8,
+ 0x2b, 0x58, 0x0a, 0xb7, 0x1b, 0x02, 0x98, 0x8e, 0x00, 0x46, 0x1b, 0xda, 0x7f, 0xaa, 0xc0, 0x20,
+ 0xbf, 0x0d, 0x0e, 0xa1, 0x69, 0x12, 0x53, 0x82, 0x1e, 0xc2, 0x40, 0x1d, 0x67, 0x1a, 0x7c, 0x4d,
+ 0xdc, 0x30, 0x88, 0x02, 0x26, 0x2e, 0x49, 0xcd, 0xe9, 0x49, 0xc5, 0x71, 0xf0, 0x35, 0x79, 0xc6,
+ 0xc5, 0x68, 0x0b, 0xea, 0x21, 0xc1, 0x3e, 0xc9, 0xc4, 0x9d, 0x69, 0x3a, 0x6a, 0x84, 0xde, 0x87,
+ 0x5e, 0x44, 0x58, 0x16, 0x78, 0xd4, 0xc5, 0xbe, 0x9f, 0x11, 0x4a, 0xd5, 0xd5, 0xe9, 0x2a, 0xf1,
+ 0x23, 0x29, 0x45, 0x9f, 0xc0, 0x50, 0x1b, 0x06, 0xfc, 0x8c, 0x5f, 0xe0, 0xd0, 0xa5, 0xc4, 0x4b,
+ 0x62, 0x9f, 0xaa, 0x7b, 0xb4, 0xa5, 0xf4, 0x07, 0x4a, 0x7d, 0x2c, 0xb5, 0xe8, 0x09, 0xf4, 0x29,
+ 0x4b, 0x32, 0x7c, 0x4a, 0xdc, 0x09, 0xf6, 0xce, 0x09, 0x9f, 0xb1, 0x26, 0xc0, 0xbb, 0x63, 0x80,
+ 0x77, 0x2c, 0x4d, 0x1e, 0x4b, 0x0b, 0xa7, 0x47, 0x4b, 0x63, 0x6a, 0x7f, 0x5b, 0x85, 0xe1, 0xaa,
+ 0x6b, 0x20, 0xea, 0x83, 0x2f, 0xb6, 0xde, 0x71, 0x2a, 0x81, 0xcf, 0xef, 0x1f, 0x87, 0x44, 0xec,
+ 0xb5, 0xe6, 0x88, 0x6f, 0xf4, 0x2e, 0x80, 0x97, 0x84, 0x21, 0xf1, 0xf8, 0x44, 0xb5, 0x49, 0x43,
+ 0xc2, 0xef, 0xa7, 0xb8, 0xf2, 0x45, 0x69, 0xa8, 0x39, 0x4d, 0x2e, 0x91, 0x55, 0xe1, 0x3e, 0xb4,
+ 0x65, 0xfa, 0x94, 0x81, 0xac, 0x0a, 0x2d, 0x29, 0x93, 0x26, 0x1f, 0x00, 0xd2, 0xc7, 0x64, 0x32,
+ 0xcb, 0x0d, 0xeb, 0xc2, 0xb0, 0xaf, 0x34, 0x8f, 0x67, 0xda, 0xfa, 0x6d, 0x68, 0x66, 0x04, 0xfb,
+ 0x6e, 0x12, 0x87, 0x33, 0x51, 0x28, 0x1a, 0x4e, 0x83, 0x0b, 0xbe, 0x8c, 0xc3, 0x19, 0xfa, 0x31,
+ 0x0c, 0x32, 0x92, 0x86, 0x81, 0x87, 0xdd, 0x34, 0xc4, 0x1e, 0x89, 0x48, 0xac, 0x6b, 0x46, 0x5f,
+ 0x29, 0x9e, 0x6b, 0x39, 0x1a, 0xc2, 0xfa, 0x05, 0xc9, 0x28, 0xdf, 0x56, 0x53, 0x98, 0xe8, 0x21,
+ 0xea, 0x43, 0x95, 0xb1, 0x70, 0x08, 0x42, 0xca, 0x3f, 0xd1, 0x03, 0xe8, 0x7b, 0x49, 0x94, 0x62,
+ 0x8f, 0xb9, 0x19, 0xb9, 0x08, 0xc4, 0xa4, 0x96, 0x50, 0xf7, 0x94, 0xdc, 0x51, 0x62, 0xbe, 0x9d,
+ 0x28, 0xf1, 0x83, 0x93, 0x80, 0xf8, 0x2e, 0x66, 0x2a, 0xd9, 0xe2, 0xe2, 0x56, 0x9d, 0xbe, 0xd6,
+ 0x3c, 0x62, 0x32, 0xcd, 0x68, 0x07, 0x36, 0x32, 0x12, 0x25, 0x8c, 0xb8, 0x3a, 0xd9, 0x31, 0x8e,
+ 0xc8, 0xb0, 0x23, 0x70, 0x1e, 0x48, 0x95, 0xca, 0xf1, 0x11, 0x8e, 0x08, 0xf7, 0x3e, 0x67, 0xcf,
+ 0x6b, 0x6d, 0x57, 0x98, 0xf7, 0x4b, 0xe6, 0x4f, 0xc9, 0xcc, 0xfe, 0x87, 0x05, 0x77, 0xaf, 0x2c,
+ 0x39, 0x0b, 0x47, 0xe0, 0xba, 0x74, 0xff, 0x50, 0x08, 0xdb, 0x53, 0xb8, 0x77, 0x4d, 0x21, 0xb8,
+ 0x26, 0xd6, 0xca, 0x42, 0xac, 0x36, 0x74, 0x88, 0xe7, 0x06, 0xb1, 0x4f, 0x2e, 0xdd, 0x49, 0xc0,
+ 0xe4, 0x15, 0xed, 0x38, 0x2d, 0xe2, 0x1d, 0x70, 0xd9, 0xe3, 0x80, 0x51, 0xfb, 0x1b, 0x0b, 0xba,
+ 0xe5, 0x3b, 0xc4, 0x6f, 0x01, 0x9b, 0xa5, 0x44, 0xf5, 0x4d, 0xf1, 0xad, 0x96, 0xae, 0xa8, 0x4e,
+ 0xea, 0xa3, 0x03, 0x80, 0x34, 0x4b, 0x52, 0x92, 0xb1, 0x80, 0x70, 0xbf, 0xfc, 0x5a, 0x3e, 0x58,
+ 0x79, 0x2d, 0x77, 0x9e, 0xe7, 0xb6, 0xfb, 0x31, 0xcb, 0x66, 0x8e, 0x31, 0x79, 0xf4, 0x29, 0xf4,
+ 0xe6, 0xd4, 0x1c, 0x1d, 0x9e, 0x55, 0x19, 0x00, 0xff, 0x44, 0x9b, 0xb0, 0x76, 0x81, 0xc3, 0x29,
+ 0x51, 0x21, 0xc8, 0xc1, 0x2f, 0x2a, 0x9f, 0x58, 0xf6, 0x3a, 0xac, 0xed, 0x47, 0x29, 0x9b, 0xf1,
+ 0x9d, 0xf4, 0x8e, 0xa7, 0x29, 0xc9, 0x1e, 0x87, 0x89, 0x77, 0xbe, 0x7f, 0xc9, 0x32, 0x8c, 0xbe,
+ 0x84, 0x2e, 0xc9, 0x30, 0x9d, 0x66, 0xfc, 0x56, 0xf9, 0x41, 0x7c, 0x2a, 0x7c, 0x96, 0x5b, 0xd2,
+ 0xdc, 0x9c, 0x9d, 0x7d, 0x39, 0x61, 0x4f, 0xd8, 0x3b, 0x1d, 0x62, 0x0e, 0x47, 0xbf, 0x85, 0x4e,
+ 0x49, 0xcf, 0xc1, 0xe2, 0x0d, 0x5c, 0x65, 0x45, 0x7c, 0xf3, 0xa2, 0x99, 0xe2, 0x2c, 0x60, 0x33,
+ 0x45, 0x34, 0xd4, 0x88, 0x97, 0x0a, 0x55, 0x78, 0x03, 0x5f, 0x82, 0xd6, 0x71, 0x9a, 0x52, 0x72,
+ 0xe0, 0x53, 0xfb, 0x21, 0x6c, 0x3e, 0x25, 0x24, 0xdd, 0x4b, 0xe2, 0x98, 0x78, 0x8c, 0xf8, 0x0e,
+ 0xf9, 0xc3, 0x94, 0x50, 0xc6, 0x97, 0x10, 0x77, 0x42, 0xe5, 0x83, 0x7f, 0xdb, 0x7f, 0xb7, 0xa0,
+ 0x2b, 0x8f, 0xcb, 0xb3, 0xc4, 0x13, 0x87, 0x84, 0x83, 0xc6, 0x19, 0x8c, 0x02, 0x6d, 0x9a, 0x85,
+ 0x73, 0xd4, 0xa6, 0x32, 0x4f, 0x6d, 0xee, 0x40, 0x43, 0xf4, 0xfe, 0x22, 0x98, 0x75, 0xde, 0xce,
+ 0x03, 0x9f, 0x16, 0x55, 0xcb, 0x97, 0xea, 0x9a, 0x50, 0xb7, 0x74, 0x7b, 0xe6, 0x26, 0x45, 0x67,
+ 0x58, 0x33, 0x3b, 0x83, 0xfd, 0x12, 0x36, 0x9e, 0x25, 0xc9, 0xf9, 0x34, 0x95, 0xe1, 0xe9, 0x4d,
+ 0x94, 0xf7, 0x6e, 0x8d, 0xab, 0x3c, 0x96, 0x7c, 0xef, 0xd7, 0x1d, 0x65, 0xfb, 0x7f, 0x16, 0x6c,
+ 0x96, 0xdd, 0xaa, 0x66, 0xf6, 0x7b, 0xd8, 0xc8, 0xfd, 0xba, 0xa1, 0xc2, 0x42, 0x2e, 0xd0, 0xda,
+ 0xfd, 0xd0, 0x48, 0xf3, 0xb2, 0xd9, 0x9a, 0x20, 0xf9, 0x1a, 0x44, 0x67, 0x70, 0x31, 0x27, 0xa1,
+ 0xa3, 0x4b, 0xe8, 0xcf, 0x9b, 0xf1, 0x22, 0x9c, 0xaf, 0xaa, 0x10, 0x6f, 0xe8, 0x99, 0xe8, 0x67,
+ 0xd0, 0x2c, 0x02, 0xa9, 0x88, 0x40, 0x36, 0x4a, 0x81, 0xa8, 0xb5, 0x0a, 0x2b, 0x7e, 0xbc, 0x49,
+ 0x96, 0x25, 0x99, 0x2a, 0x38, 0x72, 0x60, 0xff, 0x12, 0x1a, 0xdf, 0x3b, 0xbb, 0xf6, 0xbf, 0x2a,
+ 0xd0, 0x79, 0x44, 0x69, 0x70, 0x1a, 0xeb, 0x14, 0x6c, 0xc2, 0x9a, 0x6c, 0x2d, 0xb2, 0xd7, 0xcb,
+ 0x01, 0x1a, 0x43, 0x4b, 0xd5, 0x2d, 0x03, 0x7a, 0x53, 0x74, 0x6d, 0x49, 0x54, 0xb5, 0xac, 0x26,
+ 0x43, 0xe3, 0xdd, 0x62, 0x8e, 0xe8, 0xae, 0xad, 0x24, 0xba, 0x75, 0x83, 0xe8, 0xbe, 0x0d, 0x4d,
+ 0x31, 0x29, 0x4e, 0x7c, 0xa2, 0x18, 0x70, 0x83, 0x0b, 0x8e, 0x12, 0x9f, 0xa0, 0x5d, 0xd8, 0x8a,
+ 0x48, 0x94, 0x64, 0x33, 0x37, 0xc2, 0xa9, 0xcb, 0x79, 0xb6, 0xe0, 0x2e, 0xd1, 0x44, 0xd5, 0x5e,
+ 0x24, 0xb5, 0x87, 0x38, 0x3d, 0xc4, 0x97, 0x9c, 0xbe, 0x1c, 0x4e, 0xd0, 0x2e, 0xdc, 0xfe, 0x2a,
+ 0x0b, 0x18, 0x9e, 0x84, 0xa4, 0xcc, 0xdf, 0x65, 0x2d, 0xde, 0xd0, 0x4a, 0x83, 0xc4, 0xdb, 0x7f,
+ 0xb3, 0xa0, 0xab, 0x51, 0x53, 0x27, 0xac, 0x0f, 0xd5, 0x93, 0x3c, 0xcb, 0xfc, 0x53, 0xe7, 0xa2,
+ 0xb2, 0x2a, 0x17, 0x0b, 0x8f, 0x88, 0x1c, 0xf9, 0x9a, 0x89, 0x7c, 0x9e, 0xf4, 0x35, 0x23, 0xe9,
+ 0x1c, 0x1a, 0x3c, 0x65, 0x67, 0x1a, 0x1a, 0xfe, 0x6d, 0x9f, 0xc2, 0xe0, 0x98, 0x61, 0x16, 0x50,
+ 0x16, 0x78, 0x54, 0xa7, 0x73, 0x2e, 0x71, 0xd6, 0x75, 0x89, 0xab, 0xac, 0x4a, 0x5c, 0x35, 0x4f,
+ 0x9c, 0xfd, 0x1f, 0x0b, 0x90, 0xb9, 0x92, 0x82, 0xe0, 0x07, 0x58, 0x8a, 0x43, 0xc6, 0x12, 0xc6,
+ 0xd9, 0x20, 0x67, 0x5c, 0x8a, 0x37, 0x09, 0x09, 0x4f, 0x1f, 0x3f, 0x0d, 0x53, 0x4a, 0x7c, 0xa9,
+ 0x95, 0xa4, 0xa9, 0xc1, 0x05, 0x42, 0x59, 0xe6, 0x5c, 0xf5, 0x39, 0xce, 0x65, 0x3f, 0x82, 0x96,
+ 0xea, 0x3f, 0x2f, 0x79, 0xef, 0xba, 0x3e, 0x7a, 0x15, 0x5d, 0xa5, 0x00, 0x62, 0x0c, 0xb0, 0x57,
+ 0x44, 0xbf, 0xac, 0x02, 0xff, 0x11, 0x6e, 0x17, 0x16, 0xcf, 0x02, 0xca, 0x74, 0x5e, 0x3e, 0x86,
+ 0xad, 0x20, 0xf6, 0xc2, 0xa9, 0x4f, 0xdc, 0x98, 0x77, 0xf0, 0x30, 0x7f, 0xbc, 0x58, 0x82, 0xad,
+ 0x6d, 0x2a, 0xed, 0x91, 0x50, 0xea, 0x47, 0xcc, 0x07, 0x80, 0xf4, 0x2c, 0xe2, 0xe5, 0x33, 0x2a,
+ 0x62, 0x46, 0x5f, 0x69, 0xf6, 0x3d, 0x65, 0x6d, 0xbf, 0x80, 0xad, 0xf9, 0xc5, 0x55, 0xaa, 0x7e,
+ 0x0e, 0xad, 0x02, 0x76, 0x5d, 0x07, 0x6f, 0x1b, 0xe5, 0xa7, 0x98, 0xe7, 0x98, 0x96, 0xf6, 0x4f,
+ 0xe0, 0xad, 0x42, 0xf5, 0x44, 0x14, 0xfa, 0xab, 0x1a, 0xd0, 0x08, 0x86, 0x8b, 0xe6, 0x32, 0x06,
+ 0xfb, 0xaf, 0x55, 0x68, 0x3f, 0x51, 0x37, 0x97, 0xd3, 0x18, 0x83, 0xb8, 0x48, 0xf6, 0x70, 0x1f,
+ 0xda, 0xa5, 0x0b, 0x29, 0xf9, 0x76, 0xeb, 0xc2, 0x78, 0x4d, 0x2f, 0x7b, 0x77, 0x57, 0x85, 0xd9,
+ 0xfc, 0xbb, 0xfb, 0x21, 0x0c, 0x4e, 0x32, 0x42, 0x16, 0x9f, 0xe8, 0x35, 0xa7, 0xc7, 0x15, 0xa6,
+ 0xed, 0x0e, 0x6c, 0x60, 0x8f, 0x05, 0x17, 0x73, 0xd6, 0xf2, 0x7c, 0x0d, 0xa4, 0xca, 0xb4, 0xff,
+ 0x3c, 0x0f, 0x34, 0x88, 0x4f, 0x12, 0x3a, 0xac, 0x7f, 0xf7, 0x27, 0xb6, 0xda, 0x0d, 0xd7, 0x50,
+ 0xf4, 0x1c, 0xba, 0xfa, 0xa9, 0xa6, 0x3c, 0xad, 0xdf, 0xf8, 0x19, 0xd8, 0x26, 0x85, 0x8a, 0x1a,
+ 0xbc, 0xb9, 0xb4, 0x93, 0x86, 0xdc, 0x89, 0x54, 0x99, 0x85, 0xed, 0xdf, 0x15, 0x68, 0x38, 0xd8,
+ 0x3b, 0x7f, 0xb3, 0xf3, 0xf1, 0x19, 0xf4, 0xf2, 0x1e, 0x51, 0x4a, 0xc9, 0x5b, 0x06, 0x90, 0xe6,
+ 0xd1, 0x73, 0x3a, 0xbe, 0x31, 0x5a, 0x09, 0xdb, 0xfa, 0x2a, 0xd8, 0xfe, 0x59, 0x81, 0xee, 0x93,
+ 0xbc, 0x6f, 0xbd, 0xd9, 0xe0, 0xed, 0x02, 0xf0, 0x46, 0x5b, 0xc2, 0xcd, 0x24, 0x26, 0xfa, 0x78,
+ 0x38, 0xcd, 0x4c, 0x7d, 0xdd, 0x1c, 0xaf, 0x6f, 0x2a, 0xd0, 0x7e, 0x99, 0xa4, 0x49, 0x98, 0x9c,
+ 0xce, 0xde, 0x6c, 0xb4, 0xf6, 0x61, 0x60, 0x70, 0x98, 0x12, 0x68, 0x77, 0xe6, 0x0e, 0x5b, 0x71,
+ 0x38, 0x9c, 0x9e, 0x5f, 0x1a, 0xdf, 0x1c, 0xc0, 0x0d, 0x18, 0x28, 0x5e, 0x5f, 0xb4, 0x14, 0xfb,
+ 0xcf, 0x16, 0x20, 0x53, 0xaa, 0x6a, 0xfd, 0xaf, 0xa0, 0xc3, 0x14, 0xd6, 0x22, 0x3e, 0xf5, 0xb8,
+ 0x31, 0xef, 0x82, 0x99, 0x0b, 0xa7, 0xcd, 0xcc, 0xcc, 0xfc, 0x14, 0x36, 0x17, 0x7e, 0x06, 0xe2,
+ 0x84, 0x4a, 0x66, 0x64, 0x30, 0xf7, 0x4b, 0xd0, 0xe1, 0xc4, 0xfe, 0x18, 0x6e, 0x4b, 0x12, 0xad,
+ 0xfb, 0x90, 0xee, 0x0f, 0x0b, 0x6c, 0xb8, 0x53, 0xb0, 0x61, 0xfb, 0x5b, 0x0b, 0xb6, 0xe6, 0xa7,
+ 0xa9, 0xf8, 0xaf, 0x9a, 0x87, 0x30, 0x20, 0x55, 0x2f, 0x4d, 0x5e, 0x2f, 0xe9, 0xf4, 0x47, 0x0b,
+ 0xbc, 0x7e, 0xde, 0xf7, 0x8e, 0xae, 0xa3, 0x05, 0xb5, 0xef, 0xd3, 0xb2, 0x80, 0x8e, 0x30, 0x0c,
+ 0x16, 0xcc, 0xf8, 0xab, 0x48, 0xaf, 0xab, 0x62, 0x5a, 0x57, 0x13, 0xbf, 0x07, 0xb1, 0xb7, 0xef,
+ 0xc1, 0xdd, 0x2f, 0x08, 0x3b, 0x14, 0x36, 0x7b, 0x49, 0x7c, 0x12, 0x9c, 0x4e, 0x33, 0x69, 0x54,
+ 0xa4, 0xf6, 0xdd, 0x55, 0x16, 0x0a, 0xa6, 0x25, 0xbf, 0xb5, 0x59, 0x37, 0xfe, 0xad, 0xad, 0x72,
+ 0xd5, 0x6f, 0x6d, 0xbb, 0xff, 0xad, 0xc3, 0xfa, 0x31, 0xc1, 0xaf, 0x09, 0xe1, 0x4f, 0xfb, 0xce,
+ 0x31, 0x89, 0xfd, 0xe2, 0x57, 0xf4, 0x4d, 0x63, 0x8f, 0xb9, 0x74, 0xf4, 0xce, 0x32, 0x69, 0x4e,
+ 0x01, 0x6e, 0x6d, 0x5b, 0x1f, 0x5a, 0xe8, 0x05, 0x74, 0x4a, 0x2f, 0x5a, 0x74, 0xcf, 0x98, 0xb4,
+ 0xec, 0xad, 0x3b, 0xba, 0xb3, 0xd0, 0x10, 0x35, 0xaa, 0xb9, 0xcb, 0xb6, 0xf9, 0x92, 0x43, 0xef,
+ 0xae, 0x7c, 0xe2, 0x49, 0x87, 0xf7, 0xae, 0x79, 0x02, 0xda, 0xb7, 0xd0, 0x67, 0x50, 0x97, 0x94,
+ 0x1f, 0x0d, 0x0d, 0xe3, 0xd2, 0xdb, 0xa9, 0x14, 0x57, 0xf9, 0x7d, 0x60, 0xdf, 0x42, 0x4f, 0x01,
+ 0x0a, 0xd2, 0x8c, 0xde, 0x29, 0xfd, 0x0c, 0x32, 0xc7, 0xda, 0x47, 0x77, 0x57, 0x68, 0x73, 0x67,
+ 0x5f, 0x41, 0xb7, 0x4c, 0xed, 0xd0, 0x78, 0x29, 0x7b, 0x33, 0xea, 0xc3, 0xe8, 0xfe, 0x15, 0x16,
+ 0xb9, 0xe3, 0xdf, 0x41, 0x7f, 0x9e, 0xb1, 0x21, 0x7b, 0xe9, 0xc4, 0x12, 0xfb, 0x1b, 0xbd, 0x77,
+ 0xa5, 0x8d, 0x09, 0x42, 0x51, 0xa2, 0x4a, 0x20, 0x2c, 0xd4, 0xb3, 0x12, 0x08, 0x8b, 0x75, 0x4d,
+ 0x82, 0x50, 0xbe, 0xd7, 0x25, 0x10, 0x96, 0x56, 0xa1, 0x12, 0x08, 0xcb, 0x8b, 0x82, 0x7d, 0x0b,
+ 0x25, 0xb0, 0xb5, 0xfc, 0xb6, 0x21, 0xf3, 0x27, 0xa1, 0x2b, 0xaf, 0xec, 0xe8, 0xc1, 0x77, 0xb0,
+ 0xd4, 0x0b, 0x4e, 0xea, 0xe2, 0x2f, 0xaa, 0x8f, 0xfe, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x6c, 0x64,
+ 0x5c, 0xbc, 0xb2, 0x1a, 0x00, 0x00,
}
diff --git a/weed/pb/proto_read_write_test.go b/weed/pb/proto_read_write_test.go
new file mode 100644
index 000000000..7f6444ab5
--- /dev/null
+++ b/weed/pb/proto_read_write_test.go
@@ -0,0 +1,43 @@
+package pb
+
+import (
+ "fmt"
+ "testing"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/golang/protobuf/jsonpb"
+)
+
+func TestJsonpMarshalUnmarshal(t *testing.T) {
+
+ tv := &volume_server_pb.RemoteFile{
+ BackendType: "aws",
+ BackendId: "",
+ FileSize: 12,
+ }
+
+ m := jsonpb.Marshaler{
+ EmitDefaults: true,
+ Indent: " ",
+ }
+
+ if text, err := m.MarshalToString(tv); err != nil {
+ fmt.Printf("marshal eror: %v\n", err)
+ } else {
+ fmt.Printf("marshalled: %s\n", text)
+ }
+
+ rawJson := `{
+ "backendType":"aws",
+ "backendId":"temp",
+ "FileSize":12
+ }`
+
+ tv1 := &volume_server_pb.RemoteFile{}
+ if err := jsonpb.UnmarshalString(rawJson, tv1); err != nil {
+ fmt.Printf("unmarshal error: %v\n", err)
+ }
+
+ fmt.Printf("unmarshalled: %+v\n", tv1)
+
+}
diff --git a/weed/pb/volume_info.go b/weed/pb/volume_info.go
new file mode 100644
index 000000000..b2edf9c5e
--- /dev/null
+++ b/weed/pb/volume_info.go
@@ -0,0 +1,75 @@
+package pb
+
+import (
+ "bytes"
+ "fmt"
+ "io/ioutil"
+
+ _ "github.com/chrislusf/seaweedfs/weed/storage/backend/s3_backend"
+ "github.com/chrislusf/seaweedfs/weed/util"
+
+ "github.com/golang/protobuf/jsonpb"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+)
+
+// MaybeLoadVolumeInfo load the file data as *volume_server_pb.VolumeInfo, the returned volumeInfo will not be nil
+func MaybeLoadVolumeInfo(fileName string) (*volume_server_pb.VolumeInfo, bool) {
+
+ volumeInfo := &volume_server_pb.VolumeInfo{}
+
+ glog.V(1).Infof("maybeLoadVolumeInfo checks %s", fileName)
+ if exists, canRead, _, _, _ := util.CheckFile(fileName); !exists || !canRead {
+ if !exists {
+ return volumeInfo, false
+ }
+ if !canRead {
+ glog.Warningf("can not read %s", fileName)
+ }
+ return volumeInfo, false
+ }
+
+ glog.V(1).Infof("maybeLoadVolumeInfo reads %s", fileName)
+ tierData, readErr := ioutil.ReadFile(fileName)
+ if readErr != nil {
+ glog.Warningf("fail to read %s : %v", fileName, readErr)
+ return volumeInfo, false
+ }
+
+ glog.V(1).Infof("maybeLoadVolumeInfo Unmarshal volume info %v", fileName)
+ if err := jsonpb.Unmarshal(bytes.NewReader(tierData), volumeInfo); err != nil {
+ glog.Warningf("unmarshal error: %v", err)
+ return volumeInfo, false
+ }
+
+ if len(volumeInfo.GetFiles()) == 0 {
+ return volumeInfo, false
+ }
+
+ return volumeInfo, true
+}
+
+func SaveVolumeInfo(fileName string, volumeInfo *volume_server_pb.VolumeInfo) error {
+
+ if exists, _, canWrite, _, _ := util.CheckFile(fileName); exists && !canWrite {
+ return fmt.Errorf("%s not writable", fileName)
+ }
+
+ m := jsonpb.Marshaler{
+ EmitDefaults: true,
+ Indent: " ",
+ }
+
+ text, marshalErr := m.MarshalToString(volumeInfo)
+ if marshalErr != nil {
+ return fmt.Errorf("marshal to %s: %v", fileName, marshalErr)
+ }
+
+ writeErr := ioutil.WriteFile(fileName, []byte(text), 0755)
+ if writeErr != nil {
+ return fmt.Errorf("fail to write %s : %v", fileName, writeErr)
+ }
+
+ return nil
+}
diff --git a/weed/pb/volume_server.proto b/weed/pb/volume_server.proto
index 4004875ed..9cf7272ef 100644
--- a/weed/pb/volume_server.proto
+++ b/weed/pb/volume_server.proto
@@ -66,6 +66,18 @@ service VolumeServer {
}
rpc VolumeEcBlobDelete (VolumeEcBlobDeleteRequest) returns (VolumeEcBlobDeleteResponse) {
}
+ rpc VolumeEcShardsToVolume (VolumeEcShardsToVolumeRequest) returns (VolumeEcShardsToVolumeResponse) {
+ }
+
+ // tiered storage
+ rpc VolumeTierMoveDatToRemote (VolumeTierMoveDatToRemoteRequest) returns (stream VolumeTierMoveDatToRemoteResponse) {
+ }
+ rpc VolumeTierMoveDatFromRemote (VolumeTierMoveDatFromRemoteRequest) returns (stream VolumeTierMoveDatFromRemoteResponse) {
+ }
+
+ // query
+ rpc Query (QueryRequest) returns (stream QueriedStripe) {
+ }
}
@@ -127,6 +139,7 @@ message AllocateVolumeRequest {
int64 preallocate = 3;
string replication = 4;
string ttl = 5;
+ uint32 memory_map_max_size_mb = 6;
}
message AllocateVolumeResponse {
}
@@ -194,6 +207,7 @@ message CopyFileRequest {
uint64 stop_offset = 4;
string collection = 5;
bool is_ec_volume = 6;
+ bool ignore_source_file_not_found = 7;
}
message CopyFileResponse {
bytes file_content = 1;
@@ -240,6 +254,8 @@ message VolumeEcShardsCopyRequest {
repeated uint32 shard_ids = 3;
bool copy_ecx_file = 4;
string source_data_node = 5;
+ bool copy_ecj_file = 6;
+ bool copy_vif_file = 7;
}
message VolumeEcShardsCopyResponse {
}
@@ -288,6 +304,13 @@ message VolumeEcBlobDeleteRequest {
message VolumeEcBlobDeleteResponse {
}
+message VolumeEcShardsToVolumeRequest {
+ uint32 volume_id = 1;
+ string collection = 2;
+}
+message VolumeEcShardsToVolumeResponse {
+}
+
message ReadVolumeFileStatusRequest {
uint32 volume_id = 1;
}
@@ -318,3 +341,97 @@ message MemStatus {
uint64 heap = 6;
uint64 stack = 7;
}
+
+// tired storage on volume servers
+message RemoteFile {
+ string backend_type = 1;
+ string backend_id = 2;
+ string key = 3;
+ uint64 offset = 4;
+ uint64 file_size = 5;
+ uint64 modified_time = 6;
+ string extension = 7;
+}
+message VolumeInfo {
+ repeated RemoteFile files = 1;
+ uint32 version = 2;
+}
+
+message VolumeTierMoveDatToRemoteRequest {
+ uint32 volume_id = 1;
+ string collection = 2;
+ string destination_backend_name = 3;
+ bool keep_local_dat_file = 4;
+}
+message VolumeTierMoveDatToRemoteResponse {
+ int64 processed = 1;
+ float processedPercentage = 2;
+}
+
+message VolumeTierMoveDatFromRemoteRequest {
+ uint32 volume_id = 1;
+ string collection = 2;
+ bool keep_remote_dat_file = 3;
+}
+message VolumeTierMoveDatFromRemoteResponse {
+ int64 processed = 1;
+ float processedPercentage = 2;
+}
+
+// select on volume servers
+message QueryRequest {
+ repeated string selections = 1;
+ repeated string from_file_ids = 2;
+ message Filter {
+ string field = 1;
+ string operand = 2;
+ string value = 3;
+ }
+ Filter filter = 3;
+
+ message InputSerialization {
+ // NONE | GZIP | BZIP2
+ string compression_type = 1;
+ message CSVInput {
+ string file_header_info = 1; // Valid values: NONE | USE | IGNORE
+ string record_delimiter = 2; // Default: \n
+ string field_delimiter = 3; // Default: ,
+ string quote_charactoer = 4; // Default: "
+ string quote_escape_character = 5; // Default: "
+ string comments = 6; // Default: #
+ // If true, records might contain record delimiters within quote characters
+ bool allow_quoted_record_delimiter = 7; // default False.
+ }
+ message JSONInput {
+ string type = 1; // Valid values: DOCUMENT | LINES
+ }
+ message ParquetInput {
+ }
+
+ CSVInput csv_input = 2;
+ JSONInput json_input = 3;
+ ParquetInput parquet_input = 4;
+ }
+ InputSerialization input_serialization = 4;
+
+ message OutputSerialization {
+ message CSVOutput {
+ string quote_fields = 1; // Valid values: ALWAYS | ASNEEDED
+ string record_delimiter = 2; // Default: \n
+ string field_delimiter = 3; // Default: ,
+ string quote_charactoer = 4; // Default: "
+ string quote_escape_character = 5; // Default: "
+ }
+ message JSONOutput {
+ string record_delimiter = 1;
+ }
+
+ CSVOutput csv_output = 2;
+ JSONOutput json_output = 3;
+ }
+
+ OutputSerialization output_serialization = 5;
+}
+message QueriedStripe {
+ bytes records = 1;
+}
diff --git a/weed/pb/volume_server_pb/volume_server.pb.go b/weed/pb/volume_server_pb/volume_server.pb.go
index 13d14b1e5..1c2e10d8e 100644
--- a/weed/pb/volume_server_pb/volume_server.pb.go
+++ b/weed/pb/volume_server_pb/volume_server.pb.go
@@ -61,10 +61,20 @@ It has these top-level messages:
VolumeEcShardReadResponse
VolumeEcBlobDeleteRequest
VolumeEcBlobDeleteResponse
+ VolumeEcShardsToVolumeRequest
+ VolumeEcShardsToVolumeResponse
ReadVolumeFileStatusRequest
ReadVolumeFileStatusResponse
DiskStatus
MemStatus
+ RemoteFile
+ VolumeInfo
+ VolumeTierMoveDatToRemoteRequest
+ VolumeTierMoveDatToRemoteResponse
+ VolumeTierMoveDatFromRemoteRequest
+ VolumeTierMoveDatFromRemoteResponse
+ QueryRequest
+ QueriedStripe
*/
package volume_server_pb
@@ -313,11 +323,12 @@ func (*DeleteCollectionResponse) ProtoMessage() {}
func (*DeleteCollectionResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
type AllocateVolumeRequest struct {
- VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
- Collection string `protobuf:"bytes,2,opt,name=collection" json:"collection,omitempty"`
- Preallocate int64 `protobuf:"varint,3,opt,name=preallocate" json:"preallocate,omitempty"`
- Replication string `protobuf:"bytes,4,opt,name=replication" json:"replication,omitempty"`
- Ttl string `protobuf:"bytes,5,opt,name=ttl" json:"ttl,omitempty"`
+ VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
+ Collection string `protobuf:"bytes,2,opt,name=collection" json:"collection,omitempty"`
+ Preallocate int64 `protobuf:"varint,3,opt,name=preallocate" json:"preallocate,omitempty"`
+ Replication string `protobuf:"bytes,4,opt,name=replication" json:"replication,omitempty"`
+ Ttl string `protobuf:"bytes,5,opt,name=ttl" json:"ttl,omitempty"`
+ MemoryMapMaxSizeMb uint32 `protobuf:"varint,6,opt,name=memory_map_max_size_mb,json=memoryMapMaxSizeMb" json:"memory_map_max_size_mb,omitempty"`
}
func (m *AllocateVolumeRequest) Reset() { *m = AllocateVolumeRequest{} }
@@ -360,6 +371,13 @@ func (m *AllocateVolumeRequest) GetTtl() string {
return ""
}
+func (m *AllocateVolumeRequest) GetMemoryMapMaxSizeMb() uint32 {
+ if m != nil {
+ return m.MemoryMapMaxSizeMb
+ }
+ return 0
+}
+
type AllocateVolumeResponse struct {
}
@@ -649,12 +667,13 @@ func (m *VolumeCopyResponse) GetLastAppendAtNs() uint64 {
}
type CopyFileRequest struct {
- VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
- Ext string `protobuf:"bytes,2,opt,name=ext" json:"ext,omitempty"`
- CompactionRevision uint32 `protobuf:"varint,3,opt,name=compaction_revision,json=compactionRevision" json:"compaction_revision,omitempty"`
- StopOffset uint64 `protobuf:"varint,4,opt,name=stop_offset,json=stopOffset" json:"stop_offset,omitempty"`
- Collection string `protobuf:"bytes,5,opt,name=collection" json:"collection,omitempty"`
- IsEcVolume bool `protobuf:"varint,6,opt,name=is_ec_volume,json=isEcVolume" json:"is_ec_volume,omitempty"`
+ VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
+ Ext string `protobuf:"bytes,2,opt,name=ext" json:"ext,omitempty"`
+ CompactionRevision uint32 `protobuf:"varint,3,opt,name=compaction_revision,json=compactionRevision" json:"compaction_revision,omitempty"`
+ StopOffset uint64 `protobuf:"varint,4,opt,name=stop_offset,json=stopOffset" json:"stop_offset,omitempty"`
+ Collection string `protobuf:"bytes,5,opt,name=collection" json:"collection,omitempty"`
+ IsEcVolume bool `protobuf:"varint,6,opt,name=is_ec_volume,json=isEcVolume" json:"is_ec_volume,omitempty"`
+ IgnoreSourceFileNotFound bool `protobuf:"varint,7,opt,name=ignore_source_file_not_found,json=ignoreSourceFileNotFound" json:"ignore_source_file_not_found,omitempty"`
}
func (m *CopyFileRequest) Reset() { *m = CopyFileRequest{} }
@@ -704,6 +723,13 @@ func (m *CopyFileRequest) GetIsEcVolume() bool {
return false
}
+func (m *CopyFileRequest) GetIgnoreSourceFileNotFound() bool {
+ if m != nil {
+ return m.IgnoreSourceFileNotFound
+ }
+ return false
+}
+
type CopyFileResponse struct {
FileContent []byte `protobuf:"bytes,1,opt,name=file_content,json=fileContent,proto3" json:"file_content,omitempty"`
}
@@ -910,6 +936,8 @@ type VolumeEcShardsCopyRequest struct {
ShardIds []uint32 `protobuf:"varint,3,rep,packed,name=shard_ids,json=shardIds" json:"shard_ids,omitempty"`
CopyEcxFile bool `protobuf:"varint,4,opt,name=copy_ecx_file,json=copyEcxFile" json:"copy_ecx_file,omitempty"`
SourceDataNode string `protobuf:"bytes,5,opt,name=source_data_node,json=sourceDataNode" json:"source_data_node,omitempty"`
+ CopyEcjFile bool `protobuf:"varint,6,opt,name=copy_ecj_file,json=copyEcjFile" json:"copy_ecj_file,omitempty"`
+ CopyVifFile bool `protobuf:"varint,7,opt,name=copy_vif_file,json=copyVifFile" json:"copy_vif_file,omitempty"`
}
func (m *VolumeEcShardsCopyRequest) Reset() { *m = VolumeEcShardsCopyRequest{} }
@@ -952,6 +980,20 @@ func (m *VolumeEcShardsCopyRequest) GetSourceDataNode() string {
return ""
}
+func (m *VolumeEcShardsCopyRequest) GetCopyEcjFile() bool {
+ if m != nil {
+ return m.CopyEcjFile
+ }
+ return false
+}
+
+func (m *VolumeEcShardsCopyRequest) GetCopyVifFile() bool {
+ if m != nil {
+ return m.CopyVifFile
+ }
+ return false
+}
+
type VolumeEcShardsCopyResponse struct {
}
@@ -1192,6 +1234,38 @@ func (m *VolumeEcBlobDeleteResponse) String() string { return proto.C
func (*VolumeEcBlobDeleteResponse) ProtoMessage() {}
func (*VolumeEcBlobDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{51} }
+type VolumeEcShardsToVolumeRequest struct {
+ VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
+ Collection string `protobuf:"bytes,2,opt,name=collection" json:"collection,omitempty"`
+}
+
+func (m *VolumeEcShardsToVolumeRequest) Reset() { *m = VolumeEcShardsToVolumeRequest{} }
+func (m *VolumeEcShardsToVolumeRequest) String() string { return proto.CompactTextString(m) }
+func (*VolumeEcShardsToVolumeRequest) ProtoMessage() {}
+func (*VolumeEcShardsToVolumeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{52} }
+
+func (m *VolumeEcShardsToVolumeRequest) GetVolumeId() uint32 {
+ if m != nil {
+ return m.VolumeId
+ }
+ return 0
+}
+
+func (m *VolumeEcShardsToVolumeRequest) GetCollection() string {
+ if m != nil {
+ return m.Collection
+ }
+ return ""
+}
+
+type VolumeEcShardsToVolumeResponse struct {
+}
+
+func (m *VolumeEcShardsToVolumeResponse) Reset() { *m = VolumeEcShardsToVolumeResponse{} }
+func (m *VolumeEcShardsToVolumeResponse) String() string { return proto.CompactTextString(m) }
+func (*VolumeEcShardsToVolumeResponse) ProtoMessage() {}
+func (*VolumeEcShardsToVolumeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{53} }
+
type ReadVolumeFileStatusRequest struct {
VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
}
@@ -1199,7 +1273,7 @@ type ReadVolumeFileStatusRequest struct {
func (m *ReadVolumeFileStatusRequest) Reset() { *m = ReadVolumeFileStatusRequest{} }
func (m *ReadVolumeFileStatusRequest) String() string { return proto.CompactTextString(m) }
func (*ReadVolumeFileStatusRequest) ProtoMessage() {}
-func (*ReadVolumeFileStatusRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{52} }
+func (*ReadVolumeFileStatusRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{54} }
func (m *ReadVolumeFileStatusRequest) GetVolumeId() uint32 {
if m != nil {
@@ -1222,7 +1296,7 @@ type ReadVolumeFileStatusResponse struct {
func (m *ReadVolumeFileStatusResponse) Reset() { *m = ReadVolumeFileStatusResponse{} }
func (m *ReadVolumeFileStatusResponse) String() string { return proto.CompactTextString(m) }
func (*ReadVolumeFileStatusResponse) ProtoMessage() {}
-func (*ReadVolumeFileStatusResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{53} }
+func (*ReadVolumeFileStatusResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{55} }
func (m *ReadVolumeFileStatusResponse) GetVolumeId() uint32 {
if m != nil {
@@ -1290,7 +1364,7 @@ type DiskStatus struct {
func (m *DiskStatus) Reset() { *m = DiskStatus{} }
func (m *DiskStatus) String() string { return proto.CompactTextString(m) }
func (*DiskStatus) ProtoMessage() {}
-func (*DiskStatus) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{54} }
+func (*DiskStatus) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{56} }
func (m *DiskStatus) GetDir() string {
if m != nil {
@@ -1333,7 +1407,7 @@ type MemStatus struct {
func (m *MemStatus) Reset() { *m = MemStatus{} }
func (m *MemStatus) String() string { return proto.CompactTextString(m) }
func (*MemStatus) ProtoMessage() {}
-func (*MemStatus) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{55} }
+func (*MemStatus) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{57} }
func (m *MemStatus) GetGoroutines() int32 {
if m != nil {
@@ -1384,6 +1458,568 @@ func (m *MemStatus) GetStack() uint64 {
return 0
}
+// tired storage on volume servers
+type RemoteFile struct {
+ BackendType string `protobuf:"bytes,1,opt,name=backend_type,json=backendType" json:"backend_type,omitempty"`
+ BackendId string `protobuf:"bytes,2,opt,name=backend_id,json=backendId" json:"backend_id,omitempty"`
+ Key string `protobuf:"bytes,3,opt,name=key" json:"key,omitempty"`
+ Offset uint64 `protobuf:"varint,4,opt,name=offset" json:"offset,omitempty"`
+ FileSize uint64 `protobuf:"varint,5,opt,name=file_size,json=fileSize" json:"file_size,omitempty"`
+ ModifiedTime uint64 `protobuf:"varint,6,opt,name=modified_time,json=modifiedTime" json:"modified_time,omitempty"`
+ Extension string `protobuf:"bytes,7,opt,name=extension" json:"extension,omitempty"`
+}
+
+func (m *RemoteFile) Reset() { *m = RemoteFile{} }
+func (m *RemoteFile) String() string { return proto.CompactTextString(m) }
+func (*RemoteFile) ProtoMessage() {}
+func (*RemoteFile) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{58} }
+
+func (m *RemoteFile) GetBackendType() string {
+ if m != nil {
+ return m.BackendType
+ }
+ return ""
+}
+
+func (m *RemoteFile) GetBackendId() string {
+ if m != nil {
+ return m.BackendId
+ }
+ return ""
+}
+
+func (m *RemoteFile) GetKey() string {
+ if m != nil {
+ return m.Key
+ }
+ return ""
+}
+
+func (m *RemoteFile) GetOffset() uint64 {
+ if m != nil {
+ return m.Offset
+ }
+ return 0
+}
+
+func (m *RemoteFile) GetFileSize() uint64 {
+ if m != nil {
+ return m.FileSize
+ }
+ return 0
+}
+
+func (m *RemoteFile) GetModifiedTime() uint64 {
+ if m != nil {
+ return m.ModifiedTime
+ }
+ return 0
+}
+
+func (m *RemoteFile) GetExtension() string {
+ if m != nil {
+ return m.Extension
+ }
+ return ""
+}
+
+type VolumeInfo struct {
+ Files []*RemoteFile `protobuf:"bytes,1,rep,name=files" json:"files,omitempty"`
+ Version uint32 `protobuf:"varint,2,opt,name=version" json:"version,omitempty"`
+}
+
+func (m *VolumeInfo) Reset() { *m = VolumeInfo{} }
+func (m *VolumeInfo) String() string { return proto.CompactTextString(m) }
+func (*VolumeInfo) ProtoMessage() {}
+func (*VolumeInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{59} }
+
+func (m *VolumeInfo) GetFiles() []*RemoteFile {
+ if m != nil {
+ return m.Files
+ }
+ return nil
+}
+
+func (m *VolumeInfo) GetVersion() uint32 {
+ if m != nil {
+ return m.Version
+ }
+ return 0
+}
+
+type VolumeTierMoveDatToRemoteRequest struct {
+ VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
+ Collection string `protobuf:"bytes,2,opt,name=collection" json:"collection,omitempty"`
+ DestinationBackendName string `protobuf:"bytes,3,opt,name=destination_backend_name,json=destinationBackendName" json:"destination_backend_name,omitempty"`
+ KeepLocalDatFile bool `protobuf:"varint,4,opt,name=keep_local_dat_file,json=keepLocalDatFile" json:"keep_local_dat_file,omitempty"`
+}
+
+func (m *VolumeTierMoveDatToRemoteRequest) Reset() { *m = VolumeTierMoveDatToRemoteRequest{} }
+func (m *VolumeTierMoveDatToRemoteRequest) String() string { return proto.CompactTextString(m) }
+func (*VolumeTierMoveDatToRemoteRequest) ProtoMessage() {}
+func (*VolumeTierMoveDatToRemoteRequest) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{60}
+}
+
+func (m *VolumeTierMoveDatToRemoteRequest) GetVolumeId() uint32 {
+ if m != nil {
+ return m.VolumeId
+ }
+ return 0
+}
+
+func (m *VolumeTierMoveDatToRemoteRequest) GetCollection() string {
+ if m != nil {
+ return m.Collection
+ }
+ return ""
+}
+
+func (m *VolumeTierMoveDatToRemoteRequest) GetDestinationBackendName() string {
+ if m != nil {
+ return m.DestinationBackendName
+ }
+ return ""
+}
+
+func (m *VolumeTierMoveDatToRemoteRequest) GetKeepLocalDatFile() bool {
+ if m != nil {
+ return m.KeepLocalDatFile
+ }
+ return false
+}
+
+type VolumeTierMoveDatToRemoteResponse struct {
+ Processed int64 `protobuf:"varint,1,opt,name=processed" json:"processed,omitempty"`
+ ProcessedPercentage float32 `protobuf:"fixed32,2,opt,name=processedPercentage" json:"processedPercentage,omitempty"`
+}
+
+func (m *VolumeTierMoveDatToRemoteResponse) Reset() { *m = VolumeTierMoveDatToRemoteResponse{} }
+func (m *VolumeTierMoveDatToRemoteResponse) String() string { return proto.CompactTextString(m) }
+func (*VolumeTierMoveDatToRemoteResponse) ProtoMessage() {}
+func (*VolumeTierMoveDatToRemoteResponse) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{61}
+}
+
+func (m *VolumeTierMoveDatToRemoteResponse) GetProcessed() int64 {
+ if m != nil {
+ return m.Processed
+ }
+ return 0
+}
+
+func (m *VolumeTierMoveDatToRemoteResponse) GetProcessedPercentage() float32 {
+ if m != nil {
+ return m.ProcessedPercentage
+ }
+ return 0
+}
+
+type VolumeTierMoveDatFromRemoteRequest struct {
+ VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId" json:"volume_id,omitempty"`
+ Collection string `protobuf:"bytes,2,opt,name=collection" json:"collection,omitempty"`
+ KeepRemoteDatFile bool `protobuf:"varint,3,opt,name=keep_remote_dat_file,json=keepRemoteDatFile" json:"keep_remote_dat_file,omitempty"`
+}
+
+func (m *VolumeTierMoveDatFromRemoteRequest) Reset() { *m = VolumeTierMoveDatFromRemoteRequest{} }
+func (m *VolumeTierMoveDatFromRemoteRequest) String() string { return proto.CompactTextString(m) }
+func (*VolumeTierMoveDatFromRemoteRequest) ProtoMessage() {}
+func (*VolumeTierMoveDatFromRemoteRequest) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{62}
+}
+
+func (m *VolumeTierMoveDatFromRemoteRequest) GetVolumeId() uint32 {
+ if m != nil {
+ return m.VolumeId
+ }
+ return 0
+}
+
+func (m *VolumeTierMoveDatFromRemoteRequest) GetCollection() string {
+ if m != nil {
+ return m.Collection
+ }
+ return ""
+}
+
+func (m *VolumeTierMoveDatFromRemoteRequest) GetKeepRemoteDatFile() bool {
+ if m != nil {
+ return m.KeepRemoteDatFile
+ }
+ return false
+}
+
+type VolumeTierMoveDatFromRemoteResponse struct {
+ Processed int64 `protobuf:"varint,1,opt,name=processed" json:"processed,omitempty"`
+ ProcessedPercentage float32 `protobuf:"fixed32,2,opt,name=processedPercentage" json:"processedPercentage,omitempty"`
+}
+
+func (m *VolumeTierMoveDatFromRemoteResponse) Reset() { *m = VolumeTierMoveDatFromRemoteResponse{} }
+func (m *VolumeTierMoveDatFromRemoteResponse) String() string { return proto.CompactTextString(m) }
+func (*VolumeTierMoveDatFromRemoteResponse) ProtoMessage() {}
+func (*VolumeTierMoveDatFromRemoteResponse) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{63}
+}
+
+func (m *VolumeTierMoveDatFromRemoteResponse) GetProcessed() int64 {
+ if m != nil {
+ return m.Processed
+ }
+ return 0
+}
+
+func (m *VolumeTierMoveDatFromRemoteResponse) GetProcessedPercentage() float32 {
+ if m != nil {
+ return m.ProcessedPercentage
+ }
+ return 0
+}
+
+// select on volume servers
+type QueryRequest struct {
+ Selections []string `protobuf:"bytes,1,rep,name=selections" json:"selections,omitempty"`
+ FromFileIds []string `protobuf:"bytes,2,rep,name=from_file_ids,json=fromFileIds" json:"from_file_ids,omitempty"`
+ Filter *QueryRequest_Filter `protobuf:"bytes,3,opt,name=filter" json:"filter,omitempty"`
+ InputSerialization *QueryRequest_InputSerialization `protobuf:"bytes,4,opt,name=input_serialization,json=inputSerialization" json:"input_serialization,omitempty"`
+ OutputSerialization *QueryRequest_OutputSerialization `protobuf:"bytes,5,opt,name=output_serialization,json=outputSerialization" json:"output_serialization,omitempty"`
+}
+
+func (m *QueryRequest) Reset() { *m = QueryRequest{} }
+func (m *QueryRequest) String() string { return proto.CompactTextString(m) }
+func (*QueryRequest) ProtoMessage() {}
+func (*QueryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{64} }
+
+func (m *QueryRequest) GetSelections() []string {
+ if m != nil {
+ return m.Selections
+ }
+ return nil
+}
+
+func (m *QueryRequest) GetFromFileIds() []string {
+ if m != nil {
+ return m.FromFileIds
+ }
+ return nil
+}
+
+func (m *QueryRequest) GetFilter() *QueryRequest_Filter {
+ if m != nil {
+ return m.Filter
+ }
+ return nil
+}
+
+func (m *QueryRequest) GetInputSerialization() *QueryRequest_InputSerialization {
+ if m != nil {
+ return m.InputSerialization
+ }
+ return nil
+}
+
+func (m *QueryRequest) GetOutputSerialization() *QueryRequest_OutputSerialization {
+ if m != nil {
+ return m.OutputSerialization
+ }
+ return nil
+}
+
+type QueryRequest_Filter struct {
+ Field string `protobuf:"bytes,1,opt,name=field" json:"field,omitempty"`
+ Operand string `protobuf:"bytes,2,opt,name=operand" json:"operand,omitempty"`
+ Value string `protobuf:"bytes,3,opt,name=value" json:"value,omitempty"`
+}
+
+func (m *QueryRequest_Filter) Reset() { *m = QueryRequest_Filter{} }
+func (m *QueryRequest_Filter) String() string { return proto.CompactTextString(m) }
+func (*QueryRequest_Filter) ProtoMessage() {}
+func (*QueryRequest_Filter) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{64, 0} }
+
+func (m *QueryRequest_Filter) GetField() string {
+ if m != nil {
+ return m.Field
+ }
+ return ""
+}
+
+func (m *QueryRequest_Filter) GetOperand() string {
+ if m != nil {
+ return m.Operand
+ }
+ return ""
+}
+
+func (m *QueryRequest_Filter) GetValue() string {
+ if m != nil {
+ return m.Value
+ }
+ return ""
+}
+
+type QueryRequest_InputSerialization struct {
+ // NONE | GZIP | BZIP2
+ CompressionType string `protobuf:"bytes,1,opt,name=compression_type,json=compressionType" json:"compression_type,omitempty"`
+ CsvInput *QueryRequest_InputSerialization_CSVInput `protobuf:"bytes,2,opt,name=csv_input,json=csvInput" json:"csv_input,omitempty"`
+ JsonInput *QueryRequest_InputSerialization_JSONInput `protobuf:"bytes,3,opt,name=json_input,json=jsonInput" json:"json_input,omitempty"`
+ ParquetInput *QueryRequest_InputSerialization_ParquetInput `protobuf:"bytes,4,opt,name=parquet_input,json=parquetInput" json:"parquet_input,omitempty"`
+}
+
+func (m *QueryRequest_InputSerialization) Reset() { *m = QueryRequest_InputSerialization{} }
+func (m *QueryRequest_InputSerialization) String() string { return proto.CompactTextString(m) }
+func (*QueryRequest_InputSerialization) ProtoMessage() {}
+func (*QueryRequest_InputSerialization) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{64, 1}
+}
+
+func (m *QueryRequest_InputSerialization) GetCompressionType() string {
+ if m != nil {
+ return m.CompressionType
+ }
+ return ""
+}
+
+func (m *QueryRequest_InputSerialization) GetCsvInput() *QueryRequest_InputSerialization_CSVInput {
+ if m != nil {
+ return m.CsvInput
+ }
+ return nil
+}
+
+func (m *QueryRequest_InputSerialization) GetJsonInput() *QueryRequest_InputSerialization_JSONInput {
+ if m != nil {
+ return m.JsonInput
+ }
+ return nil
+}
+
+func (m *QueryRequest_InputSerialization) GetParquetInput() *QueryRequest_InputSerialization_ParquetInput {
+ if m != nil {
+ return m.ParquetInput
+ }
+ return nil
+}
+
+type QueryRequest_InputSerialization_CSVInput struct {
+ FileHeaderInfo string `protobuf:"bytes,1,opt,name=file_header_info,json=fileHeaderInfo" json:"file_header_info,omitempty"`
+ RecordDelimiter string `protobuf:"bytes,2,opt,name=record_delimiter,json=recordDelimiter" json:"record_delimiter,omitempty"`
+ FieldDelimiter string `protobuf:"bytes,3,opt,name=field_delimiter,json=fieldDelimiter" json:"field_delimiter,omitempty"`
+ QuoteCharactoer string `protobuf:"bytes,4,opt,name=quote_charactoer,json=quoteCharactoer" json:"quote_charactoer,omitempty"`
+ QuoteEscapeCharacter string `protobuf:"bytes,5,opt,name=quote_escape_character,json=quoteEscapeCharacter" json:"quote_escape_character,omitempty"`
+ Comments string `protobuf:"bytes,6,opt,name=comments" json:"comments,omitempty"`
+ // If true, records might contain record delimiters within quote characters
+ AllowQuotedRecordDelimiter bool `protobuf:"varint,7,opt,name=allow_quoted_record_delimiter,json=allowQuotedRecordDelimiter" json:"allow_quoted_record_delimiter,omitempty"`
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) Reset() {
+ *m = QueryRequest_InputSerialization_CSVInput{}
+}
+func (m *QueryRequest_InputSerialization_CSVInput) String() string { return proto.CompactTextString(m) }
+func (*QueryRequest_InputSerialization_CSVInput) ProtoMessage() {}
+func (*QueryRequest_InputSerialization_CSVInput) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{64, 1, 0}
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) GetFileHeaderInfo() string {
+ if m != nil {
+ return m.FileHeaderInfo
+ }
+ return ""
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) GetRecordDelimiter() string {
+ if m != nil {
+ return m.RecordDelimiter
+ }
+ return ""
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) GetFieldDelimiter() string {
+ if m != nil {
+ return m.FieldDelimiter
+ }
+ return ""
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) GetQuoteCharactoer() string {
+ if m != nil {
+ return m.QuoteCharactoer
+ }
+ return ""
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) GetQuoteEscapeCharacter() string {
+ if m != nil {
+ return m.QuoteEscapeCharacter
+ }
+ return ""
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) GetComments() string {
+ if m != nil {
+ return m.Comments
+ }
+ return ""
+}
+
+func (m *QueryRequest_InputSerialization_CSVInput) GetAllowQuotedRecordDelimiter() bool {
+ if m != nil {
+ return m.AllowQuotedRecordDelimiter
+ }
+ return false
+}
+
+type QueryRequest_InputSerialization_JSONInput struct {
+ Type string `protobuf:"bytes,1,opt,name=type" json:"type,omitempty"`
+}
+
+func (m *QueryRequest_InputSerialization_JSONInput) Reset() {
+ *m = QueryRequest_InputSerialization_JSONInput{}
+}
+func (m *QueryRequest_InputSerialization_JSONInput) String() string { return proto.CompactTextString(m) }
+func (*QueryRequest_InputSerialization_JSONInput) ProtoMessage() {}
+func (*QueryRequest_InputSerialization_JSONInput) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{64, 1, 1}
+}
+
+func (m *QueryRequest_InputSerialization_JSONInput) GetType() string {
+ if m != nil {
+ return m.Type
+ }
+ return ""
+}
+
+type QueryRequest_InputSerialization_ParquetInput struct {
+}
+
+func (m *QueryRequest_InputSerialization_ParquetInput) Reset() {
+ *m = QueryRequest_InputSerialization_ParquetInput{}
+}
+func (m *QueryRequest_InputSerialization_ParquetInput) String() string {
+ return proto.CompactTextString(m)
+}
+func (*QueryRequest_InputSerialization_ParquetInput) ProtoMessage() {}
+func (*QueryRequest_InputSerialization_ParquetInput) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{64, 1, 2}
+}
+
+type QueryRequest_OutputSerialization struct {
+ CsvOutput *QueryRequest_OutputSerialization_CSVOutput `protobuf:"bytes,2,opt,name=csv_output,json=csvOutput" json:"csv_output,omitempty"`
+ JsonOutput *QueryRequest_OutputSerialization_JSONOutput `protobuf:"bytes,3,opt,name=json_output,json=jsonOutput" json:"json_output,omitempty"`
+}
+
+func (m *QueryRequest_OutputSerialization) Reset() { *m = QueryRequest_OutputSerialization{} }
+func (m *QueryRequest_OutputSerialization) String() string { return proto.CompactTextString(m) }
+func (*QueryRequest_OutputSerialization) ProtoMessage() {}
+func (*QueryRequest_OutputSerialization) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{64, 2}
+}
+
+func (m *QueryRequest_OutputSerialization) GetCsvOutput() *QueryRequest_OutputSerialization_CSVOutput {
+ if m != nil {
+ return m.CsvOutput
+ }
+ return nil
+}
+
+func (m *QueryRequest_OutputSerialization) GetJsonOutput() *QueryRequest_OutputSerialization_JSONOutput {
+ if m != nil {
+ return m.JsonOutput
+ }
+ return nil
+}
+
+type QueryRequest_OutputSerialization_CSVOutput struct {
+ QuoteFields string `protobuf:"bytes,1,opt,name=quote_fields,json=quoteFields" json:"quote_fields,omitempty"`
+ RecordDelimiter string `protobuf:"bytes,2,opt,name=record_delimiter,json=recordDelimiter" json:"record_delimiter,omitempty"`
+ FieldDelimiter string `protobuf:"bytes,3,opt,name=field_delimiter,json=fieldDelimiter" json:"field_delimiter,omitempty"`
+ QuoteCharactoer string `protobuf:"bytes,4,opt,name=quote_charactoer,json=quoteCharactoer" json:"quote_charactoer,omitempty"`
+ QuoteEscapeCharacter string `protobuf:"bytes,5,opt,name=quote_escape_character,json=quoteEscapeCharacter" json:"quote_escape_character,omitempty"`
+}
+
+func (m *QueryRequest_OutputSerialization_CSVOutput) Reset() {
+ *m = QueryRequest_OutputSerialization_CSVOutput{}
+}
+func (m *QueryRequest_OutputSerialization_CSVOutput) String() string {
+ return proto.CompactTextString(m)
+}
+func (*QueryRequest_OutputSerialization_CSVOutput) ProtoMessage() {}
+func (*QueryRequest_OutputSerialization_CSVOutput) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{64, 2, 0}
+}
+
+func (m *QueryRequest_OutputSerialization_CSVOutput) GetQuoteFields() string {
+ if m != nil {
+ return m.QuoteFields
+ }
+ return ""
+}
+
+func (m *QueryRequest_OutputSerialization_CSVOutput) GetRecordDelimiter() string {
+ if m != nil {
+ return m.RecordDelimiter
+ }
+ return ""
+}
+
+func (m *QueryRequest_OutputSerialization_CSVOutput) GetFieldDelimiter() string {
+ if m != nil {
+ return m.FieldDelimiter
+ }
+ return ""
+}
+
+func (m *QueryRequest_OutputSerialization_CSVOutput) GetQuoteCharactoer() string {
+ if m != nil {
+ return m.QuoteCharactoer
+ }
+ return ""
+}
+
+func (m *QueryRequest_OutputSerialization_CSVOutput) GetQuoteEscapeCharacter() string {
+ if m != nil {
+ return m.QuoteEscapeCharacter
+ }
+ return ""
+}
+
+type QueryRequest_OutputSerialization_JSONOutput struct {
+ RecordDelimiter string `protobuf:"bytes,1,opt,name=record_delimiter,json=recordDelimiter" json:"record_delimiter,omitempty"`
+}
+
+func (m *QueryRequest_OutputSerialization_JSONOutput) Reset() {
+ *m = QueryRequest_OutputSerialization_JSONOutput{}
+}
+func (m *QueryRequest_OutputSerialization_JSONOutput) String() string {
+ return proto.CompactTextString(m)
+}
+func (*QueryRequest_OutputSerialization_JSONOutput) ProtoMessage() {}
+func (*QueryRequest_OutputSerialization_JSONOutput) Descriptor() ([]byte, []int) {
+ return fileDescriptor0, []int{64, 2, 1}
+}
+
+func (m *QueryRequest_OutputSerialization_JSONOutput) GetRecordDelimiter() string {
+ if m != nil {
+ return m.RecordDelimiter
+ }
+ return ""
+}
+
+type QueriedStripe struct {
+ Records []byte `protobuf:"bytes,1,opt,name=records,proto3" json:"records,omitempty"`
+}
+
+func (m *QueriedStripe) Reset() { *m = QueriedStripe{} }
+func (m *QueriedStripe) String() string { return proto.CompactTextString(m) }
+func (*QueriedStripe) ProtoMessage() {}
+func (*QueriedStripe) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{65} }
+
+func (m *QueriedStripe) GetRecords() []byte {
+ if m != nil {
+ return m.Records
+ }
+ return nil
+}
+
func init() {
proto.RegisterType((*BatchDeleteRequest)(nil), "volume_server_pb.BatchDeleteRequest")
proto.RegisterType((*BatchDeleteResponse)(nil), "volume_server_pb.BatchDeleteResponse")
@@ -1437,10 +2073,28 @@ func init() {
proto.RegisterType((*VolumeEcShardReadResponse)(nil), "volume_server_pb.VolumeEcShardReadResponse")
proto.RegisterType((*VolumeEcBlobDeleteRequest)(nil), "volume_server_pb.VolumeEcBlobDeleteRequest")
proto.RegisterType((*VolumeEcBlobDeleteResponse)(nil), "volume_server_pb.VolumeEcBlobDeleteResponse")
+ proto.RegisterType((*VolumeEcShardsToVolumeRequest)(nil), "volume_server_pb.VolumeEcShardsToVolumeRequest")
+ proto.RegisterType((*VolumeEcShardsToVolumeResponse)(nil), "volume_server_pb.VolumeEcShardsToVolumeResponse")
proto.RegisterType((*ReadVolumeFileStatusRequest)(nil), "volume_server_pb.ReadVolumeFileStatusRequest")
proto.RegisterType((*ReadVolumeFileStatusResponse)(nil), "volume_server_pb.ReadVolumeFileStatusResponse")
proto.RegisterType((*DiskStatus)(nil), "volume_server_pb.DiskStatus")
proto.RegisterType((*MemStatus)(nil), "volume_server_pb.MemStatus")
+ proto.RegisterType((*RemoteFile)(nil), "volume_server_pb.RemoteFile")
+ proto.RegisterType((*VolumeInfo)(nil), "volume_server_pb.VolumeInfo")
+ proto.RegisterType((*VolumeTierMoveDatToRemoteRequest)(nil), "volume_server_pb.VolumeTierMoveDatToRemoteRequest")
+ proto.RegisterType((*VolumeTierMoveDatToRemoteResponse)(nil), "volume_server_pb.VolumeTierMoveDatToRemoteResponse")
+ proto.RegisterType((*VolumeTierMoveDatFromRemoteRequest)(nil), "volume_server_pb.VolumeTierMoveDatFromRemoteRequest")
+ proto.RegisterType((*VolumeTierMoveDatFromRemoteResponse)(nil), "volume_server_pb.VolumeTierMoveDatFromRemoteResponse")
+ proto.RegisterType((*QueryRequest)(nil), "volume_server_pb.QueryRequest")
+ proto.RegisterType((*QueryRequest_Filter)(nil), "volume_server_pb.QueryRequest.Filter")
+ proto.RegisterType((*QueryRequest_InputSerialization)(nil), "volume_server_pb.QueryRequest.InputSerialization")
+ proto.RegisterType((*QueryRequest_InputSerialization_CSVInput)(nil), "volume_server_pb.QueryRequest.InputSerialization.CSVInput")
+ proto.RegisterType((*QueryRequest_InputSerialization_JSONInput)(nil), "volume_server_pb.QueryRequest.InputSerialization.JSONInput")
+ proto.RegisterType((*QueryRequest_InputSerialization_ParquetInput)(nil), "volume_server_pb.QueryRequest.InputSerialization.ParquetInput")
+ proto.RegisterType((*QueryRequest_OutputSerialization)(nil), "volume_server_pb.QueryRequest.OutputSerialization")
+ proto.RegisterType((*QueryRequest_OutputSerialization_CSVOutput)(nil), "volume_server_pb.QueryRequest.OutputSerialization.CSVOutput")
+ proto.RegisterType((*QueryRequest_OutputSerialization_JSONOutput)(nil), "volume_server_pb.QueryRequest.OutputSerialization.JSONOutput")
+ proto.RegisterType((*QueriedStripe)(nil), "volume_server_pb.QueriedStripe")
}
// Reference imports to suppress errors if they are not otherwise used.
@@ -1483,6 +2137,12 @@ type VolumeServerClient interface {
VolumeEcShardsUnmount(ctx context.Context, in *VolumeEcShardsUnmountRequest, opts ...grpc.CallOption) (*VolumeEcShardsUnmountResponse, error)
VolumeEcShardRead(ctx context.Context, in *VolumeEcShardReadRequest, opts ...grpc.CallOption) (VolumeServer_VolumeEcShardReadClient, error)
VolumeEcBlobDelete(ctx context.Context, in *VolumeEcBlobDeleteRequest, opts ...grpc.CallOption) (*VolumeEcBlobDeleteResponse, error)
+ VolumeEcShardsToVolume(ctx context.Context, in *VolumeEcShardsToVolumeRequest, opts ...grpc.CallOption) (*VolumeEcShardsToVolumeResponse, error)
+ // tiered storage
+ VolumeTierMoveDatToRemote(ctx context.Context, in *VolumeTierMoveDatToRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatToRemoteClient, error)
+ VolumeTierMoveDatFromRemote(ctx context.Context, in *VolumeTierMoveDatFromRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatFromRemoteClient, error)
+ // query
+ Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error)
}
type volumeServerClient struct {
@@ -1819,6 +2479,111 @@ func (c *volumeServerClient) VolumeEcBlobDelete(ctx context.Context, in *VolumeE
return out, nil
}
+func (c *volumeServerClient) VolumeEcShardsToVolume(ctx context.Context, in *VolumeEcShardsToVolumeRequest, opts ...grpc.CallOption) (*VolumeEcShardsToVolumeResponse, error) {
+ out := new(VolumeEcShardsToVolumeResponse)
+ err := grpc.Invoke(ctx, "/volume_server_pb.VolumeServer/VolumeEcShardsToVolume", in, out, c.cc, opts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
+func (c *volumeServerClient) VolumeTierMoveDatToRemote(ctx context.Context, in *VolumeTierMoveDatToRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatToRemoteClient, error) {
+ stream, err := grpc.NewClientStream(ctx, &_VolumeServer_serviceDesc.Streams[4], c.cc, "/volume_server_pb.VolumeServer/VolumeTierMoveDatToRemote", opts...)
+ if err != nil {
+ return nil, err
+ }
+ x := &volumeServerVolumeTierMoveDatToRemoteClient{stream}
+ if err := x.ClientStream.SendMsg(in); err != nil {
+ return nil, err
+ }
+ if err := x.ClientStream.CloseSend(); err != nil {
+ return nil, err
+ }
+ return x, nil
+}
+
+type VolumeServer_VolumeTierMoveDatToRemoteClient interface {
+ Recv() (*VolumeTierMoveDatToRemoteResponse, error)
+ grpc.ClientStream
+}
+
+type volumeServerVolumeTierMoveDatToRemoteClient struct {
+ grpc.ClientStream
+}
+
+func (x *volumeServerVolumeTierMoveDatToRemoteClient) Recv() (*VolumeTierMoveDatToRemoteResponse, error) {
+ m := new(VolumeTierMoveDatToRemoteResponse)
+ if err := x.ClientStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
+func (c *volumeServerClient) VolumeTierMoveDatFromRemote(ctx context.Context, in *VolumeTierMoveDatFromRemoteRequest, opts ...grpc.CallOption) (VolumeServer_VolumeTierMoveDatFromRemoteClient, error) {
+ stream, err := grpc.NewClientStream(ctx, &_VolumeServer_serviceDesc.Streams[5], c.cc, "/volume_server_pb.VolumeServer/VolumeTierMoveDatFromRemote", opts...)
+ if err != nil {
+ return nil, err
+ }
+ x := &volumeServerVolumeTierMoveDatFromRemoteClient{stream}
+ if err := x.ClientStream.SendMsg(in); err != nil {
+ return nil, err
+ }
+ if err := x.ClientStream.CloseSend(); err != nil {
+ return nil, err
+ }
+ return x, nil
+}
+
+type VolumeServer_VolumeTierMoveDatFromRemoteClient interface {
+ Recv() (*VolumeTierMoveDatFromRemoteResponse, error)
+ grpc.ClientStream
+}
+
+type volumeServerVolumeTierMoveDatFromRemoteClient struct {
+ grpc.ClientStream
+}
+
+func (x *volumeServerVolumeTierMoveDatFromRemoteClient) Recv() (*VolumeTierMoveDatFromRemoteResponse, error) {
+ m := new(VolumeTierMoveDatFromRemoteResponse)
+ if err := x.ClientStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
+func (c *volumeServerClient) Query(ctx context.Context, in *QueryRequest, opts ...grpc.CallOption) (VolumeServer_QueryClient, error) {
+ stream, err := grpc.NewClientStream(ctx, &_VolumeServer_serviceDesc.Streams[6], c.cc, "/volume_server_pb.VolumeServer/Query", opts...)
+ if err != nil {
+ return nil, err
+ }
+ x := &volumeServerQueryClient{stream}
+ if err := x.ClientStream.SendMsg(in); err != nil {
+ return nil, err
+ }
+ if err := x.ClientStream.CloseSend(); err != nil {
+ return nil, err
+ }
+ return x, nil
+}
+
+type VolumeServer_QueryClient interface {
+ Recv() (*QueriedStripe, error)
+ grpc.ClientStream
+}
+
+type volumeServerQueryClient struct {
+ grpc.ClientStream
+}
+
+func (x *volumeServerQueryClient) Recv() (*QueriedStripe, error) {
+ m := new(QueriedStripe)
+ if err := x.ClientStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
// Server API for VolumeServer service
type VolumeServerServer interface {
@@ -1851,6 +2616,12 @@ type VolumeServerServer interface {
VolumeEcShardsUnmount(context.Context, *VolumeEcShardsUnmountRequest) (*VolumeEcShardsUnmountResponse, error)
VolumeEcShardRead(*VolumeEcShardReadRequest, VolumeServer_VolumeEcShardReadServer) error
VolumeEcBlobDelete(context.Context, *VolumeEcBlobDeleteRequest) (*VolumeEcBlobDeleteResponse, error)
+ VolumeEcShardsToVolume(context.Context, *VolumeEcShardsToVolumeRequest) (*VolumeEcShardsToVolumeResponse, error)
+ // tiered storage
+ VolumeTierMoveDatToRemote(*VolumeTierMoveDatToRemoteRequest, VolumeServer_VolumeTierMoveDatToRemoteServer) error
+ VolumeTierMoveDatFromRemote(*VolumeTierMoveDatFromRemoteRequest, VolumeServer_VolumeTierMoveDatFromRemoteServer) error
+ // query
+ Query(*QueryRequest, VolumeServer_QueryServer) error
}
func RegisterVolumeServerServer(s *grpc.Server, srv VolumeServerServer) {
@@ -2337,6 +3108,87 @@ func _VolumeServer_VolumeEcBlobDelete_Handler(srv interface{}, ctx context.Conte
return interceptor(ctx, in, info, handler)
}
+func _VolumeServer_VolumeEcShardsToVolume_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(VolumeEcShardsToVolumeRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(VolumeServerServer).VolumeEcShardsToVolume(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: "/volume_server_pb.VolumeServer/VolumeEcShardsToVolume",
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(VolumeServerServer).VolumeEcShardsToVolume(ctx, req.(*VolumeEcShardsToVolumeRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
+func _VolumeServer_VolumeTierMoveDatToRemote_Handler(srv interface{}, stream grpc.ServerStream) error {
+ m := new(VolumeTierMoveDatToRemoteRequest)
+ if err := stream.RecvMsg(m); err != nil {
+ return err
+ }
+ return srv.(VolumeServerServer).VolumeTierMoveDatToRemote(m, &volumeServerVolumeTierMoveDatToRemoteServer{stream})
+}
+
+type VolumeServer_VolumeTierMoveDatToRemoteServer interface {
+ Send(*VolumeTierMoveDatToRemoteResponse) error
+ grpc.ServerStream
+}
+
+type volumeServerVolumeTierMoveDatToRemoteServer struct {
+ grpc.ServerStream
+}
+
+func (x *volumeServerVolumeTierMoveDatToRemoteServer) Send(m *VolumeTierMoveDatToRemoteResponse) error {
+ return x.ServerStream.SendMsg(m)
+}
+
+func _VolumeServer_VolumeTierMoveDatFromRemote_Handler(srv interface{}, stream grpc.ServerStream) error {
+ m := new(VolumeTierMoveDatFromRemoteRequest)
+ if err := stream.RecvMsg(m); err != nil {
+ return err
+ }
+ return srv.(VolumeServerServer).VolumeTierMoveDatFromRemote(m, &volumeServerVolumeTierMoveDatFromRemoteServer{stream})
+}
+
+type VolumeServer_VolumeTierMoveDatFromRemoteServer interface {
+ Send(*VolumeTierMoveDatFromRemoteResponse) error
+ grpc.ServerStream
+}
+
+type volumeServerVolumeTierMoveDatFromRemoteServer struct {
+ grpc.ServerStream
+}
+
+func (x *volumeServerVolumeTierMoveDatFromRemoteServer) Send(m *VolumeTierMoveDatFromRemoteResponse) error {
+ return x.ServerStream.SendMsg(m)
+}
+
+func _VolumeServer_Query_Handler(srv interface{}, stream grpc.ServerStream) error {
+ m := new(QueryRequest)
+ if err := stream.RecvMsg(m); err != nil {
+ return err
+ }
+ return srv.(VolumeServerServer).Query(m, &volumeServerQueryServer{stream})
+}
+
+type VolumeServer_QueryServer interface {
+ Send(*QueriedStripe) error
+ grpc.ServerStream
+}
+
+type volumeServerQueryServer struct {
+ grpc.ServerStream
+}
+
+func (x *volumeServerQueryServer) Send(m *QueriedStripe) error {
+ return x.ServerStream.SendMsg(m)
+}
+
var _VolumeServer_serviceDesc = grpc.ServiceDesc{
ServiceName: "volume_server_pb.VolumeServer",
HandlerType: (*VolumeServerServer)(nil),
@@ -2429,6 +3281,10 @@ var _VolumeServer_serviceDesc = grpc.ServiceDesc{
MethodName: "VolumeEcBlobDelete",
Handler: _VolumeServer_VolumeEcBlobDelete_Handler,
},
+ {
+ MethodName: "VolumeEcShardsToVolume",
+ Handler: _VolumeServer_VolumeEcShardsToVolume_Handler,
+ },
},
Streams: []grpc.StreamDesc{
{
@@ -2451,6 +3307,21 @@ var _VolumeServer_serviceDesc = grpc.ServiceDesc{
Handler: _VolumeServer_VolumeEcShardRead_Handler,
ServerStreams: true,
},
+ {
+ StreamName: "VolumeTierMoveDatToRemote",
+ Handler: _VolumeServer_VolumeTierMoveDatToRemote_Handler,
+ ServerStreams: true,
+ },
+ {
+ StreamName: "VolumeTierMoveDatFromRemote",
+ Handler: _VolumeServer_VolumeTierMoveDatFromRemote_Handler,
+ ServerStreams: true,
+ },
+ {
+ StreamName: "Query",
+ Handler: _VolumeServer_Query_Handler,
+ ServerStreams: true,
+ },
},
Metadata: "volume_server.proto",
}
@@ -2458,126 +3329,187 @@ var _VolumeServer_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("volume_server.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{
- // 1924 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xc4, 0x19, 0xcb, 0x72, 0xdc, 0xc6,
- 0x91, 0xd0, 0x2e, 0xb9, 0xbb, 0xbd, 0x4b, 0x89, 0x1c, 0x52, 0xe4, 0x0a, 0x14, 0x29, 0x1a, 0x76,
- 0x6c, 0x8a, 0xb2, 0x49, 0x45, 0xae, 0x24, 0x4e, 0x72, 0x48, 0x44, 0x8a, 0x49, 0x54, 0x8e, 0xe9,
- 0x2a, 0x50, 0x56, 0x39, 0x65, 0x57, 0xa1, 0x86, 0xc0, 0x50, 0x44, 0x11, 0x0b, 0x40, 0x98, 0x01,
- 0xa5, 0x55, 0x25, 0x27, 0xe5, 0x9a, 0x0f, 0xc8, 0x39, 0xb7, 0x1c, 0x72, 0xcd, 0x07, 0xe4, 0x17,
- 0x72, 0xcb, 0x37, 0xe4, 0x0b, 0x72, 0x49, 0xcd, 0x03, 0x58, 0x3c, 0x77, 0xc1, 0x90, 0x55, 0xb9,
- 0x61, 0x7b, 0xfa, 0x3d, 0xdd, 0x3d, 0xdd, 0xbd, 0xb0, 0x72, 0x19, 0x78, 0xf1, 0x88, 0x58, 0x94,
- 0x44, 0x97, 0x24, 0xda, 0x0b, 0xa3, 0x80, 0x05, 0x68, 0x29, 0x07, 0xb4, 0xc2, 0x53, 0x63, 0x1f,
- 0xd0, 0x01, 0x66, 0xf6, 0xf9, 0x33, 0xe2, 0x11, 0x46, 0x4c, 0xf2, 0x3a, 0x26, 0x94, 0xa1, 0x7b,
- 0xd0, 0x3d, 0x73, 0x3d, 0x62, 0xb9, 0x0e, 0x1d, 0x6a, 0xdb, 0xad, 0x9d, 0x9e, 0xd9, 0xe1, 0xbf,
- 0x9f, 0x3b, 0xd4, 0xf8, 0x1a, 0x56, 0x72, 0x04, 0x34, 0x0c, 0x7c, 0x4a, 0xd0, 0x17, 0xd0, 0x89,
- 0x08, 0x8d, 0x3d, 0x26, 0x09, 0xfa, 0x4f, 0xb6, 0xf6, 0x8a, 0xb2, 0xf6, 0x52, 0x92, 0xd8, 0x63,
- 0x66, 0x82, 0x6e, 0xbc, 0xd7, 0x60, 0x90, 0x3d, 0x41, 0xeb, 0xd0, 0x51, 0xc2, 0x87, 0xda, 0xb6,
- 0xb6, 0xd3, 0x33, 0x17, 0xa4, 0x6c, 0xb4, 0x06, 0x0b, 0x94, 0x61, 0x16, 0xd3, 0xe1, 0xad, 0x6d,
- 0x6d, 0x67, 0xde, 0x54, 0xbf, 0xd0, 0x2a, 0xcc, 0x93, 0x28, 0x0a, 0xa2, 0x61, 0x4b, 0xa0, 0xcb,
- 0x1f, 0x08, 0x41, 0x9b, 0xba, 0xef, 0xc8, 0xb0, 0xbd, 0xad, 0xed, 0x2c, 0x9a, 0xe2, 0x1b, 0x0d,
- 0xa1, 0x73, 0x49, 0x22, 0xea, 0x06, 0xfe, 0x70, 0x5e, 0x80, 0x93, 0x9f, 0x46, 0x07, 0xe6, 0x8f,
- 0x46, 0x21, 0x1b, 0x1b, 0x3f, 0x81, 0xe1, 0x4b, 0x6c, 0xc7, 0xf1, 0xe8, 0xa5, 0x50, 0xff, 0xf0,
- 0x9c, 0xd8, 0x17, 0x89, 0x5b, 0x36, 0xa0, 0xa7, 0x8c, 0x52, 0xba, 0x2d, 0x9a, 0x5d, 0x09, 0x78,
- 0xee, 0x18, 0xbf, 0x84, 0x7b, 0x15, 0x84, 0xca, 0x3d, 0x1f, 0xc2, 0xe2, 0x2b, 0x1c, 0x9d, 0xe2,
- 0x57, 0xc4, 0x8a, 0x30, 0x73, 0x03, 0x41, 0xad, 0x99, 0x03, 0x05, 0x34, 0x39, 0xcc, 0xf8, 0x0e,
- 0xf4, 0x1c, 0x87, 0x60, 0x14, 0x62, 0x9b, 0x35, 0x11, 0x8e, 0xb6, 0xa1, 0x1f, 0x46, 0x04, 0x7b,
- 0x5e, 0x60, 0x63, 0x46, 0x84, 0x7f, 0x5a, 0x66, 0x16, 0x64, 0x6c, 0xc2, 0x46, 0x25, 0x73, 0xa9,
- 0xa0, 0xf1, 0x45, 0x41, 0xfb, 0x60, 0x34, 0x72, 0x1b, 0x89, 0x36, 0xee, 0x97, 0xb4, 0x16, 0x94,
- 0x8a, 0xef, 0x4f, 0x0b, 0xa7, 0x1e, 0xc1, 0x7e, 0x1c, 0x36, 0x62, 0x5c, 0xd4, 0x38, 0x21, 0x4d,
- 0x39, 0xaf, 0xcb, 0xb0, 0x39, 0x0c, 0x3c, 0x8f, 0xd8, 0xcc, 0x0d, 0xfc, 0x84, 0xed, 0x16, 0x80,
- 0x9d, 0x02, 0x55, 0x10, 0x65, 0x20, 0x86, 0x0e, 0xc3, 0x32, 0xa9, 0x62, 0xfb, 0x57, 0x0d, 0xee,
- 0x3e, 0x55, 0x4e, 0x93, 0x82, 0x1b, 0x5d, 0x40, 0x5e, 0xe4, 0xad, 0xa2, 0xc8, 0xe2, 0x05, 0xb5,
- 0x4a, 0x17, 0xc4, 0x31, 0x22, 0x12, 0x7a, 0xae, 0x8d, 0x05, 0x8b, 0xb6, 0x60, 0x91, 0x05, 0xa1,
- 0x25, 0x68, 0x31, 0xe6, 0x89, 0xc8, 0xed, 0x99, 0xfc, 0xd3, 0x18, 0xc2, 0x5a, 0x51, 0x57, 0x65,
- 0xc6, 0x8f, 0x61, 0x5d, 0x42, 0x4e, 0xc6, 0xbe, 0x7d, 0x22, 0xf2, 0xa4, 0x91, 0xd3, 0xff, 0xa3,
- 0xc1, 0xb0, 0x4c, 0xa8, 0xa2, 0xf8, 0xba, 0x1e, 0xb8, 0xaa, 0x7d, 0xe8, 0x01, 0xf4, 0x19, 0x76,
- 0x3d, 0x2b, 0x38, 0x3b, 0xa3, 0x84, 0x0d, 0x17, 0xb6, 0xb5, 0x9d, 0xb6, 0x09, 0x1c, 0xf4, 0xb5,
- 0x80, 0xa0, 0x87, 0xb0, 0x64, 0xcb, 0x48, 0xb6, 0x22, 0x72, 0xe9, 0x8a, 0xcc, 0xee, 0x08, 0xc5,
- 0xee, 0xd8, 0x49, 0x84, 0x4b, 0x30, 0x32, 0x60, 0xd1, 0x75, 0xde, 0x5a, 0xa2, 0xb4, 0x88, 0xc2,
- 0xd0, 0x15, 0xdc, 0xfa, 0xae, 0xf3, 0xf6, 0x57, 0xae, 0x47, 0x4e, 0xdc, 0x77, 0xc4, 0x78, 0x09,
- 0xf7, 0xa5, 0xf1, 0xcf, 0x7d, 0x3b, 0x22, 0x23, 0xe2, 0x33, 0xec, 0x1d, 0x06, 0xe1, 0xb8, 0x51,
- 0x08, 0xdc, 0x83, 0x2e, 0x75, 0x7d, 0x9b, 0x58, 0xbe, 0x2c, 0x50, 0x6d, 0xb3, 0x23, 0x7e, 0x1f,
- 0x53, 0xe3, 0x00, 0x36, 0x6b, 0xf8, 0x2a, 0xcf, 0x7e, 0x00, 0x03, 0xa1, 0x98, 0x1d, 0xf8, 0x8c,
- 0xf8, 0x4c, 0xf0, 0x1e, 0x98, 0x7d, 0x0e, 0x3b, 0x94, 0x20, 0xe3, 0x87, 0x80, 0x24, 0x8f, 0xaf,
- 0x82, 0xd8, 0x6f, 0x96, 0x9a, 0x77, 0x61, 0x25, 0x47, 0xa2, 0x62, 0xe3, 0x73, 0x58, 0x95, 0xe0,
- 0x6f, 0xfc, 0x51, 0x63, 0x5e, 0xeb, 0x70, 0xb7, 0x40, 0xa4, 0xb8, 0x3d, 0x49, 0x84, 0xe4, 0x9f,
- 0x90, 0xa9, 0xcc, 0xd6, 0x12, 0x0d, 0xf2, 0xaf, 0x88, 0xa8, 0x42, 0x52, 0x61, 0x1c, 0x5d, 0x98,
- 0x04, 0x3b, 0x81, 0xef, 0x8d, 0x1b, 0x57, 0xa1, 0x0a, 0x4a, 0xc5, 0xf7, 0x6f, 0x1a, 0x2c, 0x27,
- 0xe5, 0xa9, 0xe1, 0x6d, 0x5e, 0x31, 0x9c, 0x5b, 0xb5, 0xe1, 0xdc, 0x9e, 0x84, 0xf3, 0x0e, 0x2c,
- 0xd1, 0x20, 0x8e, 0x6c, 0x62, 0x39, 0x98, 0x61, 0xcb, 0x0f, 0x1c, 0xa2, 0xa2, 0xfd, 0xb6, 0x84,
- 0x3f, 0xc3, 0x0c, 0x1f, 0x07, 0x0e, 0x31, 0x7e, 0x91, 0x5c, 0x76, 0x2e, 0x4a, 0x1e, 0xc2, 0xb2,
- 0x87, 0x29, 0xb3, 0x70, 0x18, 0x12, 0xdf, 0xb1, 0x30, 0xe3, 0xa1, 0xa6, 0x89, 0x50, 0xbb, 0xcd,
- 0x0f, 0x9e, 0x0a, 0xf8, 0x53, 0x76, 0x4c, 0x8d, 0x7f, 0x6a, 0x70, 0x87, 0xd3, 0xf2, 0xd0, 0x6e,
- 0x64, 0xef, 0x12, 0xb4, 0xc8, 0x5b, 0xa6, 0x0c, 0xe5, 0x9f, 0x68, 0x1f, 0x56, 0x54, 0x0e, 0xb9,
- 0x81, 0x3f, 0x49, 0xaf, 0x96, 0x20, 0x44, 0x93, 0xa3, 0x34, 0xc3, 0x1e, 0x40, 0x9f, 0xb2, 0x20,
- 0x4c, 0xb2, 0xb5, 0x2d, 0xb3, 0x95, 0x83, 0x54, 0xb6, 0xe6, 0x7d, 0x3a, 0x5f, 0xe1, 0xd3, 0x81,
- 0x4b, 0x2d, 0x62, 0x5b, 0x52, 0x2b, 0x91, 0xef, 0x5d, 0x13, 0x5c, 0x7a, 0x64, 0x4b, 0x6f, 0x18,
- 0x3f, 0x82, 0xa5, 0x89, 0x55, 0xcd, 0x73, 0xe7, 0xbd, 0x96, 0x94, 0xc3, 0x17, 0xd8, 0xf5, 0x4e,
- 0x88, 0xef, 0x90, 0xe8, 0x9a, 0x39, 0x8d, 0x1e, 0xc3, 0xaa, 0xeb, 0x78, 0xc4, 0x62, 0xee, 0x88,
- 0x04, 0x31, 0xb3, 0x28, 0xb1, 0x03, 0xdf, 0xa1, 0x89, 0x7f, 0xf8, 0xd9, 0x0b, 0x79, 0x74, 0x22,
- 0x4f, 0x8c, 0x3f, 0xa6, 0xb5, 0x35, 0xab, 0xc5, 0xa4, 0x43, 0xf0, 0x09, 0xe1, 0x0c, 0xcf, 0x09,
- 0x76, 0x48, 0xa4, 0xcc, 0x18, 0x48, 0xe0, 0x6f, 0x04, 0x8c, 0x7b, 0x58, 0x21, 0x9d, 0x06, 0xce,
- 0x58, 0x68, 0x34, 0x30, 0x41, 0x82, 0x0e, 0x02, 0x67, 0x2c, 0x8a, 0x1c, 0xb5, 0x44, 0x90, 0xd8,
- 0xe7, 0xb1, 0x7f, 0x21, 0xb4, 0xe9, 0x9a, 0x7d, 0x97, 0xfe, 0x16, 0x53, 0x76, 0xc8, 0x41, 0xc6,
- 0xdf, 0xb5, 0x24, 0xcb, 0xb8, 0x1a, 0x26, 0xb1, 0x89, 0x7b, 0xf9, 0x7f, 0x70, 0x07, 0xa7, 0x50,
- 0xd9, 0x90, 0xeb, 0x14, 0x55, 0xc2, 0x20, 0x79, 0xa6, 0xde, 0x22, 0x71, 0x32, 0x49, 0xf2, 0xbc,
- 0xe2, 0x2a, 0xc9, 0xbf, 0x4f, 0x8a, 0xec, 0x91, 0x7d, 0x72, 0x8e, 0x23, 0x87, 0xfe, 0x9a, 0xf8,
- 0x24, 0xc2, 0xec, 0x46, 0x1e, 0x70, 0x63, 0x1b, 0xb6, 0xea, 0xb8, 0x2b, 0xf9, 0xdf, 0x25, 0x8f,
- 0x47, 0x82, 0x61, 0x92, 0xd3, 0xd8, 0xf5, 0x9c, 0x1b, 0x11, 0xff, 0x65, 0xd1, 0xb8, 0x94, 0xb9,
- 0x8a, 0x9f, 0x5d, 0x58, 0x8e, 0x04, 0x88, 0x59, 0x94, 0x23, 0xa4, 0xbd, 0xfb, 0xa2, 0x79, 0x47,
- 0x1d, 0x08, 0x42, 0xde, 0xc3, 0xff, 0x23, 0x8d, 0x80, 0x84, 0xdb, 0x8d, 0x95, 0xc5, 0x0d, 0xe8,
- 0x4d, 0xc4, 0xb7, 0x84, 0xf8, 0x2e, 0x55, 0x72, 0x79, 0x74, 0xda, 0x41, 0x38, 0xb6, 0x88, 0x2d,
- 0xdf, 0x61, 0x71, 0xd5, 0x5d, 0xb3, 0xcf, 0x81, 0x47, 0xb6, 0x78, 0x86, 0xaf, 0x50, 0x23, 0xd3,
- 0x68, 0xc8, 0x1b, 0xa1, 0x6e, 0xe3, 0x0d, 0x6c, 0xe4, 0x4f, 0x9b, 0x3f, 0x4f, 0xd7, 0x32, 0xd2,
- 0xd8, 0x2a, 0x86, 0x41, 0xe1, 0x8d, 0xbb, 0x2c, 0xaa, 0xdd, 0xf8, 0x3d, 0xbf, 0x9e, 0x5e, 0x9b,
- 0x45, 0x87, 0xe4, 0x9b, 0x82, 0x6f, 0x8b, 0x6a, 0x5f, 0xa1, 0x39, 0x98, 0x2e, 0xf8, 0x41, 0x31,
- 0x74, 0x8b, 0x1d, 0xc4, 0x9f, 0xd3, 0xba, 0xa8, 0x30, 0xf8, 0xfb, 0xdd, 0xb8, 0x1e, 0x29, 0xb9,
- 0xc2, 0x1d, 0x8b, 0x66, 0x47, 0x89, 0xe5, 0xc3, 0xa2, 0x7a, 0x87, 0x64, 0xaf, 0xad, 0x7e, 0xe5,
- 0xc6, 0xc2, 0x96, 0x1a, 0x0b, 0x93, 0x71, 0xf7, 0x82, 0x8c, 0x45, 0xac, 0xb5, 0xe5, 0xb8, 0xfb,
- 0x25, 0x19, 0x1b, 0xc7, 0x85, 0x4c, 0x91, 0xaa, 0xa9, 0x9c, 0x43, 0xd0, 0xe6, 0x41, 0xaa, 0x4a,
- 0xb5, 0xf8, 0x46, 0x9b, 0x00, 0x2e, 0xb5, 0x1c, 0x71, 0xe7, 0x52, 0xa9, 0xae, 0xd9, 0x73, 0x55,
- 0x10, 0x38, 0xc6, 0x9f, 0x32, 0xa9, 0x77, 0xe0, 0x05, 0xa7, 0x37, 0x18, 0x95, 0x59, 0x2b, 0x5a,
- 0x39, 0x2b, 0xb2, 0x73, 0x6f, 0x3b, 0x3f, 0xf7, 0x66, 0x92, 0x28, 0xab, 0x8e, 0xba, 0x99, 0x9f,
- 0xc1, 0x06, 0x37, 0x58, 0x62, 0x88, 0x2e, 0xb9, 0xf9, 0x24, 0xf1, 0xef, 0x5b, 0x70, 0xbf, 0x9a,
- 0xb8, 0xc9, 0x34, 0xf1, 0x73, 0xd0, 0xd3, 0x6e, 0x9d, 0x3f, 0x29, 0x94, 0xe1, 0x51, 0x98, 0x3e,
- 0x2a, 0xf2, 0xed, 0x59, 0x57, 0xad, 0xfb, 0x8b, 0xe4, 0x3c, 0x79, 0x59, 0x4a, 0xad, 0x7e, 0xab,
- 0xd4, 0xea, 0x73, 0x01, 0x0e, 0x66, 0x75, 0x02, 0x64, 0xef, 0xb2, 0xee, 0x60, 0x56, 0x27, 0x20,
- 0x25, 0x16, 0x02, 0x64, 0xd4, 0xf4, 0x15, 0xbe, 0x10, 0xb0, 0x09, 0xa0, 0xda, 0x92, 0xd8, 0x4f,
- 0x46, 0x97, 0x9e, 0x6c, 0x4a, 0x62, 0xbf, 0xb6, 0xbb, 0xea, 0xd4, 0x76, 0x57, 0xf9, 0xeb, 0xef,
- 0x96, 0x5e, 0x88, 0x6f, 0x01, 0x9e, 0xb9, 0xf4, 0x42, 0x3a, 0x99, 0xb7, 0x73, 0x8e, 0x1b, 0xa9,
- 0xd9, 0x97, 0x7f, 0x72, 0x08, 0xf6, 0x3c, 0xe5, 0x3a, 0xfe, 0xc9, 0xc3, 0x37, 0xa6, 0xc4, 0x51,
- 0xde, 0x11, 0xdf, 0x1c, 0x76, 0x16, 0x11, 0xa2, 0x1c, 0x20, 0xbe, 0x8d, 0xbf, 0x68, 0xd0, 0xfb,
- 0x8a, 0x8c, 0x14, 0xe7, 0x2d, 0x80, 0x57, 0x41, 0x14, 0xc4, 0xcc, 0xf5, 0x89, 0xec, 0x3e, 0xe7,
- 0xcd, 0x0c, 0xe4, 0x7f, 0x97, 0x23, 0x52, 0x93, 0x78, 0x67, 0xca, 0x99, 0xe2, 0x9b, 0xc3, 0xce,
- 0x09, 0x0e, 0x95, 0xff, 0xc4, 0x37, 0x5a, 0x85, 0x79, 0xca, 0xb0, 0x7d, 0x21, 0x9c, 0xd5, 0x36,
- 0xe5, 0x8f, 0x27, 0xff, 0x5a, 0x83, 0x41, 0xb6, 0x5b, 0x40, 0xdf, 0x43, 0x3f, 0xb3, 0xa9, 0x42,
- 0x1f, 0x95, 0x17, 0x52, 0xe5, 0xcd, 0x97, 0xfe, 0x83, 0x19, 0x58, 0x2a, 0x31, 0xe6, 0x90, 0x0f,
- 0xcb, 0xa5, 0x75, 0x0f, 0xda, 0x2d, 0x53, 0xd7, 0x2d, 0x93, 0xf4, 0x47, 0x8d, 0x70, 0x53, 0x79,
- 0x0c, 0x56, 0x2a, 0xf6, 0x37, 0xe8, 0xd3, 0x19, 0x5c, 0x72, 0x3b, 0x24, 0xfd, 0xb3, 0x86, 0xd8,
- 0xa9, 0xd4, 0xd7, 0x80, 0xca, 0xcb, 0x1d, 0xf4, 0x68, 0x26, 0x9b, 0xc9, 0xf2, 0x48, 0xff, 0xb4,
- 0x19, 0x72, 0xad, 0xa1, 0x72, 0xed, 0x33, 0xd3, 0xd0, 0xdc, 0x62, 0x69, 0xa6, 0xa1, 0x85, 0x5d,
- 0xd2, 0x1c, 0xba, 0x80, 0xa5, 0xe2, 0x4a, 0x08, 0x3d, 0xac, 0x5b, 0x61, 0x96, 0x36, 0x4e, 0xfa,
- 0x6e, 0x13, 0xd4, 0x54, 0x18, 0x81, 0xdb, 0xf9, 0xb5, 0x0d, 0xfa, 0xa4, 0x4c, 0x5f, 0xb9, 0x84,
- 0xd2, 0x77, 0x66, 0x23, 0x66, 0x6d, 0x2a, 0xae, 0x72, 0xaa, 0x6c, 0xaa, 0xd9, 0x13, 0x55, 0xd9,
- 0x54, 0xb7, 0x19, 0x32, 0xe6, 0xd0, 0xef, 0x93, 0xfd, 0x40, 0x61, 0xc5, 0x81, 0xf6, 0xea, 0xd8,
- 0x54, 0xef, 0x58, 0xf4, 0xfd, 0xc6, 0xf8, 0x89, 0xec, 0xc7, 0x1a, 0xcf, 0xf5, 0xcc, 0xa6, 0xa3,
- 0x2a, 0xd7, 0xcb, 0xbb, 0x93, 0xaa, 0x5c, 0xaf, 0x5a, 0x97, 0xcc, 0xa1, 0x53, 0x58, 0xcc, 0xed,
- 0x3e, 0xd0, 0xc7, 0x75, 0x94, 0xf9, 0xa6, 0x49, 0xff, 0x64, 0x26, 0x5e, 0x2a, 0xc3, 0x4a, 0xaa,
- 0x97, 0x2a, 0x57, 0xb5, 0xca, 0xe5, 0xeb, 0xd5, 0xc7, 0xb3, 0xd0, 0x72, 0xa9, 0x5c, 0xda, 0x90,
- 0x54, 0xa6, 0x72, 0xdd, 0x06, 0xa6, 0x32, 0x95, 0xeb, 0x97, 0x2e, 0x73, 0xe8, 0x77, 0x00, 0x93,
- 0x2d, 0x06, 0xfa, 0xb0, 0x8e, 0x3a, 0x7b, 0xfb, 0x1f, 0x4d, 0x47, 0x4a, 0x59, 0xbf, 0x81, 0xd5,
- 0xaa, 0xe6, 0x02, 0x55, 0x24, 0xfe, 0x94, 0x0e, 0x46, 0xdf, 0x6b, 0x8a, 0x9e, 0x0a, 0xfe, 0x06,
- 0xba, 0xc9, 0x06, 0x02, 0x7d, 0x50, 0xa6, 0x2e, 0xec, 0x5c, 0x74, 0x63, 0x1a, 0x4a, 0x26, 0x80,
- 0x47, 0x49, 0xae, 0x4e, 0x56, 0x03, 0xf5, 0xb9, 0x5a, 0x5a, 0x62, 0xd4, 0xe7, 0x6a, 0x79, 0xd3,
- 0x20, 0xc4, 0xa5, 0xc1, 0x90, 0x9d, 0xa4, 0xeb, 0x83, 0xa1, 0x62, 0x51, 0x50, 0x1f, 0x0c, 0x95,
- 0xc3, 0xf9, 0x1c, 0xfa, 0x03, 0xac, 0x55, 0x0f, 0xd0, 0xa8, 0x36, 0xe3, 0x6b, 0x06, 0x79, 0xfd,
- 0x71, 0x73, 0x82, 0x54, 0xfc, 0xbb, 0xa4, 0x3e, 0x15, 0x06, 0xe8, 0xfa, 0xfa, 0x54, 0x3d, 0xc6,
- 0xeb, 0xfb, 0x8d, 0xf1, 0xcb, 0xa9, 0x97, 0x9d, 0x54, 0xeb, 0xbd, 0x5d, 0x31, 0x94, 0xd7, 0x7b,
- 0xbb, 0x72, 0xf8, 0x15, 0xf9, 0x51, 0x35, 0x85, 0x56, 0xe5, 0xc7, 0x94, 0x31, 0x59, 0xdf, 0x6b,
- 0x8a, 0x9e, 0x7b, 0xbe, 0xcb, 0x63, 0x26, 0x9a, 0xa9, 0x7f, 0xae, 0x32, 0x7f, 0xd6, 0x10, 0xbb,
- 0xfe, 0x76, 0x93, 0x4a, 0x3d, 0xd3, 0x80, 0x42, 0xc5, 0xde, 0x6f, 0x8c, 0x9f, 0xca, 0x0e, 0x93,
- 0xdd, 0x72, 0x66, 0x44, 0x44, 0xbb, 0x33, 0xf8, 0x64, 0x46, 0x5c, 0xfd, 0x51, 0x23, 0xdc, 0xaa,
- 0xec, 0xcd, 0x0e, 0x6d, 0xd3, 0xe2, 0xa9, 0x34, 0x69, 0x4e, 0x8b, 0xa7, 0x8a, 0x39, 0x70, 0xee,
- 0x74, 0x41, 0xfc, 0x81, 0xfc, 0xf9, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xc8, 0x44, 0x1a, 0xc5,
- 0x57, 0x1e, 0x00, 0x00,
+ // 2905 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xd4, 0x3a, 0x4d, 0x73, 0xdc, 0xc6,
+ 0xb1, 0x5c, 0x2e, 0x3f, 0x76, 0x7b, 0x49, 0x91, 0x1a, 0xd2, 0xd4, 0x1a, 0xa2, 0x24, 0x1a, 0xf2,
+ 0x87, 0x24, 0x5b, 0x94, 0x4c, 0xdb, 0xcf, 0x7e, 0xf6, 0xb3, 0xdf, 0x93, 0x28, 0xe9, 0x45, 0xb1,
+ 0x45, 0xd9, 0xa0, 0xac, 0x38, 0xb6, 0x2b, 0xa8, 0x21, 0x30, 0x2b, 0xc2, 0x04, 0x30, 0x10, 0x30,
+ 0x4b, 0x6b, 0x55, 0xc9, 0xc9, 0x39, 0xa4, 0x2a, 0x95, 0x1c, 0x52, 0xb9, 0xe4, 0x9c, 0x7b, 0xae,
+ 0xf9, 0x03, 0x39, 0xf8, 0x0f, 0xa4, 0x2a, 0xa7, 0x5c, 0x72, 0xce, 0x21, 0xb7, 0x54, 0xe5, 0x92,
+ 0x9a, 0x2f, 0x2c, 0x3e, 0xb9, 0xa0, 0xc5, 0x54, 0x2a, 0xb7, 0x41, 0x4f, 0x7f, 0x4c, 0xf7, 0x74,
+ 0xf7, 0x4c, 0x4f, 0x03, 0x56, 0x0e, 0xa9, 0x3f, 0x0c, 0x88, 0x9d, 0x90, 0xf8, 0x90, 0xc4, 0x9b,
+ 0x51, 0x4c, 0x19, 0x45, 0xcb, 0x39, 0xa0, 0x1d, 0xed, 0x99, 0xd7, 0x00, 0xdd, 0xc4, 0xcc, 0xd9,
+ 0xbf, 0x45, 0x7c, 0xc2, 0x88, 0x45, 0x1e, 0x0f, 0x49, 0xc2, 0xd0, 0xf3, 0xd0, 0x19, 0x78, 0x3e,
+ 0xb1, 0x3d, 0x37, 0xe9, 0xb7, 0x36, 0xda, 0x97, 0xba, 0xd6, 0x3c, 0xff, 0xbe, 0xeb, 0x26, 0xe6,
+ 0x7d, 0x58, 0xc9, 0x11, 0x24, 0x11, 0x0d, 0x13, 0x82, 0xde, 0x81, 0xf9, 0x98, 0x24, 0x43, 0x9f,
+ 0x49, 0x82, 0xde, 0xd6, 0xf9, 0xcd, 0xa2, 0xac, 0xcd, 0x94, 0x64, 0xe8, 0x33, 0x4b, 0xa3, 0x9b,
+ 0xdf, 0xb4, 0x60, 0x21, 0x3b, 0x83, 0xce, 0xc0, 0xbc, 0x12, 0xde, 0x6f, 0x6d, 0xb4, 0x2e, 0x75,
+ 0xad, 0x39, 0x29, 0x1b, 0xad, 0xc1, 0x5c, 0xc2, 0x30, 0x1b, 0x26, 0xfd, 0xe9, 0x8d, 0xd6, 0xa5,
+ 0x59, 0x4b, 0x7d, 0xa1, 0x55, 0x98, 0x25, 0x71, 0x4c, 0xe3, 0x7e, 0x5b, 0xa0, 0xcb, 0x0f, 0x84,
+ 0x60, 0x26, 0xf1, 0x9e, 0x92, 0xfe, 0xcc, 0x46, 0xeb, 0xd2, 0xa2, 0x25, 0xc6, 0xa8, 0x0f, 0xf3,
+ 0x87, 0x24, 0x4e, 0x3c, 0x1a, 0xf6, 0x67, 0x05, 0x58, 0x7f, 0x9a, 0xf3, 0x30, 0x7b, 0x3b, 0x88,
+ 0xd8, 0xc8, 0x7c, 0x1b, 0xfa, 0x0f, 0xb1, 0x33, 0x1c, 0x06, 0x0f, 0xc5, 0xf2, 0xb7, 0xf7, 0x89,
+ 0x73, 0xa0, 0xcd, 0x72, 0x16, 0xba, 0x4a, 0x29, 0xb5, 0xb6, 0x45, 0xab, 0x23, 0x01, 0x77, 0x5d,
+ 0xf3, 0xff, 0xe0, 0xf9, 0x0a, 0x42, 0x65, 0x9e, 0x8b, 0xb0, 0xf8, 0x08, 0xc7, 0x7b, 0xf8, 0x11,
+ 0xb1, 0x63, 0xcc, 0x3c, 0x2a, 0xa8, 0x5b, 0xd6, 0x82, 0x02, 0x5a, 0x1c, 0x66, 0x7e, 0x01, 0x46,
+ 0x8e, 0x03, 0x0d, 0x22, 0xec, 0xb0, 0x26, 0xc2, 0xd1, 0x06, 0xf4, 0xa2, 0x98, 0x60, 0xdf, 0xa7,
+ 0x0e, 0x66, 0x44, 0xd8, 0xa7, 0x6d, 0x65, 0x41, 0xe6, 0x39, 0x38, 0x5b, 0xc9, 0x5c, 0x2e, 0xd0,
+ 0x7c, 0xa7, 0xb0, 0x7a, 0x1a, 0x04, 0x5e, 0x23, 0xd1, 0xe6, 0x7a, 0x69, 0xd5, 0x82, 0x52, 0xf1,
+ 0xfd, 0xef, 0xc2, 0xac, 0x4f, 0x70, 0x38, 0x8c, 0x1a, 0x31, 0x2e, 0xae, 0x58, 0x93, 0xa6, 0x9c,
+ 0xcf, 0x48, 0xb7, 0xd9, 0xa6, 0xbe, 0x4f, 0x1c, 0xe6, 0xd1, 0x50, 0xb3, 0x3d, 0x0f, 0xe0, 0xa4,
+ 0x40, 0xe5, 0x44, 0x19, 0x88, 0x69, 0x40, 0xbf, 0x4c, 0xaa, 0xd8, 0xfe, 0xb9, 0x05, 0xcf, 0xdd,
+ 0x50, 0x46, 0x93, 0x82, 0x1b, 0x6d, 0x40, 0x5e, 0xe4, 0x74, 0x51, 0x64, 0x71, 0x83, 0xda, 0xa5,
+ 0x0d, 0xe2, 0x18, 0x31, 0x89, 0x7c, 0xcf, 0xc1, 0x82, 0xc5, 0x8c, 0x60, 0x91, 0x05, 0xa1, 0x65,
+ 0x68, 0x33, 0xe6, 0x0b, 0xcf, 0xed, 0x5a, 0x7c, 0x88, 0xb6, 0x60, 0x2d, 0x20, 0x01, 0x8d, 0x47,
+ 0x76, 0x80, 0x23, 0x3b, 0xc0, 0x4f, 0x6c, 0xee, 0xe6, 0x76, 0xb0, 0xd7, 0x9f, 0x13, 0xeb, 0x43,
+ 0x72, 0xf6, 0x1e, 0x8e, 0xee, 0xe1, 0x27, 0xbb, 0xde, 0x53, 0x72, 0x6f, 0xcf, 0xec, 0xc3, 0x5a,
+ 0x51, 0x3f, 0xa5, 0xfa, 0x7f, 0xc1, 0x19, 0x09, 0xd9, 0x1d, 0x85, 0xce, 0xae, 0x88, 0xad, 0x46,
+ 0x1b, 0xf5, 0x8f, 0x16, 0xf4, 0xcb, 0x84, 0xca, 0xf3, 0x9f, 0xd5, 0x6a, 0xc7, 0xb6, 0xc9, 0x05,
+ 0xe8, 0x31, 0xec, 0xf9, 0x36, 0x1d, 0x0c, 0x12, 0xc2, 0x84, 0x21, 0x66, 0x2c, 0xe0, 0xa0, 0xfb,
+ 0x02, 0x82, 0x2e, 0xc3, 0xb2, 0x23, 0xbd, 0xdf, 0x8e, 0xc9, 0xa1, 0x27, 0xb2, 0xc1, 0xbc, 0x58,
+ 0xd8, 0x92, 0xa3, 0xa3, 0x42, 0x82, 0x91, 0x09, 0x8b, 0x9e, 0xfb, 0xc4, 0x16, 0xe9, 0x48, 0x24,
+ 0x93, 0x8e, 0xe0, 0xd6, 0xf3, 0xdc, 0x27, 0x77, 0x3c, 0x9f, 0x70, 0x8b, 0x9a, 0x0f, 0x61, 0x5d,
+ 0x2a, 0x7f, 0x37, 0x74, 0x62, 0x12, 0x90, 0x90, 0x61, 0x7f, 0x9b, 0x46, 0xa3, 0x46, 0x6e, 0xf3,
+ 0x3c, 0x74, 0x12, 0x2f, 0x74, 0x88, 0x1d, 0xca, 0xa4, 0x36, 0x63, 0xcd, 0x8b, 0xef, 0x9d, 0xc4,
+ 0xbc, 0x09, 0xe7, 0x6a, 0xf8, 0x2a, 0xcb, 0xbe, 0x00, 0x0b, 0x62, 0x61, 0x0e, 0x0d, 0x19, 0x09,
+ 0x99, 0xe0, 0xbd, 0x60, 0xf5, 0x38, 0x6c, 0x5b, 0x82, 0xcc, 0xd7, 0x01, 0x49, 0x1e, 0xf7, 0xe8,
+ 0x30, 0x6c, 0x16, 0xce, 0xcf, 0xc1, 0x4a, 0x8e, 0x44, 0xf9, 0xc6, 0x1b, 0xb0, 0x2a, 0xc1, 0x9f,
+ 0x86, 0x41, 0x63, 0x5e, 0x67, 0xe0, 0xb9, 0x02, 0x91, 0xe2, 0xb6, 0xa5, 0x85, 0xe4, 0x8f, 0x9d,
+ 0x23, 0x99, 0xad, 0xe9, 0x15, 0xe4, 0x4f, 0x1e, 0x91, 0xb9, 0xe4, 0x82, 0x71, 0x7c, 0x60, 0x11,
+ 0xec, 0xd2, 0xd0, 0x1f, 0x35, 0xce, 0x5c, 0x15, 0x94, 0x8a, 0xef, 0xef, 0x5a, 0x70, 0x5a, 0xa7,
+ 0xb4, 0x86, 0xbb, 0x79, 0x4c, 0x77, 0x6e, 0xd7, 0xba, 0xf3, 0xcc, 0xd8, 0x9d, 0x2f, 0xc1, 0x72,
+ 0x42, 0x87, 0xb1, 0x43, 0x6c, 0x17, 0x33, 0x6c, 0x87, 0xd4, 0x25, 0xca, 0xdb, 0x4f, 0x49, 0xf8,
+ 0x2d, 0xcc, 0xf0, 0x0e, 0x75, 0x89, 0xf9, 0xbf, 0x7a, 0xb3, 0x73, 0x5e, 0x72, 0x19, 0x4e, 0xfb,
+ 0x38, 0x61, 0x36, 0x8e, 0x22, 0x12, 0xba, 0x36, 0x66, 0xdc, 0xd5, 0x5a, 0xc2, 0xd5, 0x4e, 0xf1,
+ 0x89, 0x1b, 0x02, 0x7e, 0x83, 0xed, 0x24, 0xe6, 0xaf, 0xa7, 0x61, 0x89, 0xd3, 0x72, 0xd7, 0x6e,
+ 0xa4, 0xef, 0x32, 0xb4, 0xc9, 0x13, 0xa6, 0x14, 0xe5, 0x43, 0x74, 0x0d, 0x56, 0x54, 0x0c, 0x79,
+ 0x34, 0x1c, 0x87, 0x57, 0x5b, 0x66, 0xa3, 0xf1, 0x54, 0x1a, 0x61, 0x17, 0xa0, 0x97, 0x30, 0x1a,
+ 0xe9, 0x68, 0x9d, 0x91, 0xd1, 0xca, 0x41, 0x2a, 0x5a, 0xf3, 0x36, 0x9d, 0xad, 0xb0, 0xe9, 0x82,
+ 0x97, 0xd8, 0xc4, 0xb1, 0xe5, 0xaa, 0x44, 0xbc, 0x77, 0x2c, 0xf0, 0x92, 0xdb, 0x8e, 0xb4, 0x06,
+ 0xfa, 0x00, 0xd6, 0xbd, 0x47, 0x21, 0x8d, 0x89, 0xad, 0x0c, 0x29, 0xa2, 0x26, 0xa4, 0xcc, 0x1e,
+ 0xd0, 0x61, 0xe8, 0x8a, 0xd8, 0xef, 0x58, 0x7d, 0x89, 0xb3, 0x2b, 0x50, 0xb8, 0x05, 0x76, 0x28,
+ 0xbb, 0xc3, 0xe7, 0xcd, 0xb7, 0x60, 0x79, 0x6c, 0x95, 0xe6, 0xb1, 0xf7, 0x4d, 0x4b, 0xa7, 0xd3,
+ 0x07, 0xd8, 0xf3, 0x77, 0x49, 0xe8, 0x92, 0xf8, 0x19, 0x73, 0x02, 0xba, 0x0e, 0xab, 0x9e, 0xeb,
+ 0x13, 0x9b, 0x79, 0x01, 0xa1, 0x43, 0x66, 0x27, 0xc4, 0xa1, 0xa1, 0x9b, 0x68, 0xfb, 0xf2, 0xb9,
+ 0x07, 0x72, 0x6a, 0x57, 0xce, 0x98, 0x3f, 0x4d, 0x73, 0x73, 0x76, 0x15, 0xe3, 0x5b, 0x49, 0x48,
+ 0x08, 0x67, 0xb8, 0x4f, 0xb0, 0x4b, 0x62, 0xa5, 0xc6, 0x82, 0x04, 0x7e, 0x4f, 0xc0, 0xf8, 0x0e,
+ 0x29, 0xa4, 0x3d, 0xea, 0x8e, 0xc4, 0x8a, 0x16, 0x2c, 0x90, 0xa0, 0x9b, 0xd4, 0x1d, 0x89, 0x24,
+ 0x99, 0xd8, 0xc2, 0xc9, 0x9c, 0xfd, 0x61, 0x78, 0x20, 0x56, 0xd3, 0xb1, 0x7a, 0x5e, 0xf2, 0x11,
+ 0x4e, 0xd8, 0x36, 0x07, 0x99, 0xbf, 0x6f, 0xe9, 0x28, 0xe5, 0xcb, 0xb0, 0x88, 0x43, 0xbc, 0xc3,
+ 0x7f, 0x83, 0x39, 0x38, 0x85, 0x72, 0x82, 0xdc, 0xed, 0x54, 0x05, 0x1c, 0x92, 0x73, 0xea, 0x2c,
+ 0x13, 0x33, 0xe3, 0x24, 0x91, 0x5f, 0xb8, 0x4a, 0x12, 0x5f, 0xea, 0x24, 0x7d, 0xdb, 0xd9, 0xdd,
+ 0xc7, 0xb1, 0x9b, 0xfc, 0x3f, 0x09, 0x49, 0x8c, 0xd9, 0x89, 0x5c, 0x1a, 0xcc, 0x0d, 0x38, 0x5f,
+ 0xc7, 0x5d, 0xc9, 0xff, 0x42, 0x1f, 0x3e, 0x1a, 0xc3, 0x22, 0x7b, 0x43, 0xcf, 0x77, 0x4f, 0x44,
+ 0xfc, 0x87, 0x45, 0xe5, 0x52, 0xe6, 0xca, 0x7f, 0xae, 0xc0, 0xe9, 0x58, 0x80, 0x98, 0x9d, 0x70,
+ 0x84, 0xb4, 0x5e, 0x58, 0xb4, 0x96, 0xd4, 0x84, 0x20, 0xe4, 0x75, 0xc3, 0xcf, 0xa7, 0xb5, 0x07,
+ 0x68, 0x6e, 0x27, 0x96, 0x56, 0xcf, 0x42, 0x77, 0x2c, 0xbe, 0x2d, 0xc4, 0x77, 0x12, 0x25, 0x97,
+ 0x7b, 0xa7, 0x43, 0xa3, 0x91, 0x4d, 0x1c, 0x79, 0x8e, 0x8b, 0xad, 0xee, 0x58, 0x3d, 0x0e, 0xbc,
+ 0xed, 0x88, 0x63, 0xbc, 0x79, 0x8e, 0xcd, 0x70, 0xfb, 0x4a, 0x72, 0x9b, 0xcb, 0x72, 0xfb, 0x4a,
+ 0x70, 0xd3, 0x38, 0x87, 0xde, 0x40, 0xe2, 0xcc, 0x8f, 0x71, 0x1e, 0x7a, 0x03, 0x8e, 0x33, 0xf6,
+ 0xaa, 0xbc, 0x31, 0xd4, 0xae, 0x7e, 0x0d, 0x67, 0xf3, 0xb3, 0xcd, 0x8f, 0xc9, 0x67, 0x32, 0x96,
+ 0x79, 0xbe, 0xe8, 0x4e, 0x85, 0xb3, 0xf6, 0xb0, 0xb8, 0xec, 0xc6, 0xf7, 0x8a, 0x67, 0x5b, 0xd7,
+ 0xb9, 0xa2, 0x41, 0xf2, 0x97, 0x93, 0xcf, 0x8a, 0xcb, 0x3e, 0xc6, 0x25, 0xe5, 0x68, 0xc1, 0x17,
+ 0x8a, 0x21, 0x50, 0xbc, 0xc9, 0xfc, 0x26, 0xcd, 0xaf, 0x0a, 0x83, 0xdf, 0x23, 0x1a, 0xe7, 0x35,
+ 0x25, 0x57, 0x98, 0x63, 0xd1, 0x9a, 0x57, 0x62, 0x79, 0xa1, 0xab, 0xce, 0x43, 0x59, 0x27, 0xa8,
+ 0xaf, 0x5c, 0x49, 0xdb, 0x56, 0x25, 0xad, 0x2e, 0xd5, 0x0f, 0xc8, 0x48, 0xf8, 0xec, 0x8c, 0x2c,
+ 0xd5, 0x3f, 0x24, 0x23, 0x73, 0xa7, 0x10, 0x71, 0x72, 0x69, 0x2a, 0x76, 0x11, 0xcc, 0x70, 0x67,
+ 0x57, 0x29, 0x5f, 0x8c, 0xd1, 0x39, 0x00, 0x2f, 0xb1, 0x5d, 0xb1, 0xe7, 0x72, 0x51, 0x1d, 0xab,
+ 0xeb, 0x29, 0x27, 0x70, 0xcd, 0x5f, 0xb4, 0xc6, 0x0c, 0x6f, 0xfa, 0x74, 0xef, 0x04, 0xbd, 0x32,
+ 0xab, 0x45, 0x3b, 0xa7, 0x45, 0xb6, 0x66, 0x9f, 0xc9, 0xd7, 0xec, 0x99, 0x20, 0xca, 0x2e, 0xa7,
+ 0x2e, 0x35, 0x3f, 0xa0, 0x27, 0x57, 0xcf, 0x95, 0x53, 0xf3, 0x98, 0xbb, 0x92, 0xff, 0x2e, 0x9c,
+ 0xe5, 0x06, 0x97, 0x50, 0x51, 0x2d, 0x34, 0xaf, 0xa8, 0xfe, 0x3a, 0x0d, 0xeb, 0xd5, 0xc4, 0x4d,
+ 0xaa, 0xaa, 0xf7, 0xc0, 0x48, 0xab, 0x16, 0x7e, 0x34, 0x26, 0x0c, 0x07, 0x51, 0x7a, 0x38, 0xca,
+ 0x33, 0xf4, 0x8c, 0x2a, 0x61, 0x1e, 0xe8, 0x79, 0x7d, 0x42, 0x96, 0x4a, 0x9e, 0x76, 0xa9, 0xe4,
+ 0xe1, 0x02, 0x5c, 0xcc, 0xea, 0x04, 0xc8, 0x3b, 0xdc, 0x19, 0x17, 0xb3, 0x3a, 0x01, 0x29, 0xb1,
+ 0x10, 0x20, 0xbd, 0xb6, 0xa7, 0xf0, 0x85, 0x80, 0x73, 0x00, 0xea, 0x7a, 0x35, 0x0c, 0x75, 0x09,
+ 0xd7, 0x95, 0x97, 0xab, 0x61, 0x58, 0x7b, 0xcb, 0x9c, 0xaf, 0xbd, 0x65, 0xe6, 0x77, 0xb3, 0x53,
+ 0xda, 0xcd, 0xcf, 0x00, 0x6e, 0x79, 0xc9, 0x81, 0x34, 0x32, 0xbf, 0xd6, 0xba, 0x5e, 0xac, 0xde,
+ 0x0d, 0xf8, 0x90, 0x43, 0xb0, 0xef, 0x2b, 0xd3, 0xf1, 0x21, 0x0f, 0x9f, 0x61, 0x42, 0x5c, 0x65,
+ 0x1d, 0x31, 0xe6, 0xb0, 0x41, 0x4c, 0x88, 0x32, 0x80, 0x18, 0x9b, 0xbf, 0x6d, 0x41, 0xf7, 0x1e,
+ 0x09, 0x14, 0xe7, 0xf3, 0x00, 0x8f, 0x68, 0x4c, 0x87, 0xcc, 0x0b, 0x89, 0xbc, 0x85, 0xcf, 0x5a,
+ 0x19, 0xc8, 0x77, 0x97, 0x23, 0x52, 0x03, 0xf1, 0x07, 0xca, 0x98, 0x62, 0xcc, 0x61, 0xfb, 0x04,
+ 0x47, 0xca, 0x7e, 0x62, 0x8c, 0x56, 0x61, 0x36, 0x61, 0xd8, 0x39, 0x10, 0xc6, 0x9a, 0xb1, 0xe4,
+ 0x87, 0xf9, 0xa7, 0x16, 0x80, 0x45, 0x02, 0xca, 0x84, 0xaf, 0xf1, 0xdb, 0xed, 0x1e, 0x76, 0x0e,
+ 0x78, 0xbd, 0xc0, 0x46, 0x11, 0x51, 0x96, 0xe8, 0x29, 0xd8, 0x83, 0x51, 0x24, 0x76, 0x48, 0xa3,
+ 0xa8, 0xfc, 0xd5, 0xb5, 0xba, 0x0a, 0x22, 0x2b, 0x03, 0x1d, 0xca, 0x5d, 0x8b, 0x0f, 0x33, 0x39,
+ 0x4d, 0x2e, 0x5b, 0xe7, 0xb4, 0xb3, 0xd0, 0x2d, 0xba, 0x82, 0x48, 0x05, 0xc2, 0x0f, 0x2e, 0xc2,
+ 0x62, 0x40, 0x5d, 0x6f, 0xe0, 0x11, 0x57, 0x38, 0x9a, 0x52, 0x65, 0x41, 0x03, 0xb9, 0x73, 0xa1,
+ 0x75, 0xe8, 0x92, 0x27, 0x8c, 0x84, 0xa9, 0x0f, 0x74, 0xad, 0x31, 0xc0, 0xfc, 0x1c, 0x40, 0x97,
+ 0xd1, 0x03, 0x8a, 0xb6, 0x60, 0x96, 0x33, 0xd7, 0x8f, 0x94, 0xeb, 0xe5, 0x47, 0xca, 0xb1, 0x19,
+ 0x2c, 0x89, 0x9a, 0x4d, 0x40, 0xd3, 0xf9, 0x04, 0xf4, 0x6d, 0x0b, 0x36, 0xd4, 0xe5, 0xd0, 0x23,
+ 0xf1, 0x3d, 0x7a, 0xc8, 0x2f, 0x0a, 0x0f, 0xa8, 0x64, 0x71, 0x22, 0x79, 0xf1, 0x1d, 0xe8, 0xbb,
+ 0x24, 0x61, 0x5e, 0x28, 0xca, 0x43, 0x5b, 0x9b, 0x3c, 0xc4, 0x01, 0x51, 0xc6, 0x5d, 0xcb, 0xcc,
+ 0xdf, 0x94, 0xd3, 0x3b, 0x38, 0x20, 0xe8, 0x2a, 0xac, 0x1c, 0x10, 0x12, 0xd9, 0x3e, 0x75, 0xb0,
+ 0x6f, 0xeb, 0x88, 0x53, 0xb7, 0x9f, 0x65, 0x3e, 0xf5, 0x11, 0x9f, 0xb9, 0x25, 0xa3, 0xce, 0x4c,
+ 0xe0, 0x85, 0x23, 0x34, 0x51, 0x59, 0x67, 0x1d, 0xba, 0x51, 0x4c, 0x1d, 0x92, 0x70, 0x8f, 0x6c,
+ 0x89, 0x43, 0x68, 0x0c, 0x40, 0xd7, 0x61, 0x25, 0xfd, 0xf8, 0x98, 0xc4, 0x0e, 0x09, 0x19, 0x7e,
+ 0x24, 0xdf, 0x22, 0xa7, 0xad, 0xaa, 0x29, 0xf3, 0x57, 0x2d, 0x30, 0x4b, 0x52, 0xef, 0xc4, 0x34,
+ 0x38, 0x41, 0x0b, 0x5e, 0x83, 0x55, 0x61, 0x87, 0x58, 0xb0, 0x1c, 0x1b, 0x42, 0x16, 0x29, 0xa7,
+ 0xf9, 0x9c, 0x94, 0xa6, 0x2d, 0x31, 0x84, 0x8b, 0x47, 0xae, 0xe9, 0x5f, 0x64, 0x8b, 0xbf, 0x2f,
+ 0xc0, 0xc2, 0x27, 0x43, 0x12, 0x8f, 0x32, 0x8f, 0x98, 0x09, 0x51, 0x5a, 0xe8, 0x57, 0xf8, 0x0c,
+ 0x84, 0xe7, 0xd1, 0x41, 0x4c, 0x03, 0x3b, 0x7d, 0xa8, 0x9f, 0x16, 0x28, 0x3d, 0x0e, 0xbc, 0x23,
+ 0x1f, 0xeb, 0xd1, 0xfb, 0x30, 0x37, 0xf0, 0x7c, 0x46, 0xe4, 0xd3, 0x78, 0x6f, 0xeb, 0xa5, 0xb2,
+ 0xbf, 0x67, 0x65, 0x6e, 0xde, 0x11, 0xc8, 0x96, 0x22, 0x42, 0x7b, 0xb0, 0xe2, 0x85, 0x91, 0x28,
+ 0xac, 0x62, 0x0f, 0xfb, 0xde, 0xd3, 0xf1, 0x33, 0x5c, 0x6f, 0xeb, 0xf5, 0x09, 0xbc, 0xee, 0x72,
+ 0xca, 0xdd, 0x2c, 0xa1, 0x85, 0xbc, 0x12, 0x0c, 0x11, 0x58, 0xa5, 0x43, 0x56, 0x16, 0x32, 0x2b,
+ 0x84, 0x6c, 0x4d, 0x10, 0x72, 0x5f, 0x90, 0xe6, 0xa5, 0xac, 0xd0, 0x32, 0xd0, 0xd8, 0x81, 0x39,
+ 0xa9, 0x1c, 0xcf, 0x80, 0x03, 0x8f, 0xf8, 0xba, 0xb9, 0x20, 0x3f, 0x78, 0x90, 0xd3, 0x88, 0xc4,
+ 0x38, 0xd4, 0xc9, 0x4c, 0x7f, 0x72, 0xfc, 0x43, 0xec, 0x0f, 0x75, 0xbc, 0xc9, 0x0f, 0xe3, 0x8f,
+ 0xb3, 0x80, 0xca, 0x1a, 0xea, 0xb7, 0xc5, 0x98, 0x24, 0x3c, 0x41, 0x64, 0xb3, 0xe7, 0x52, 0x06,
+ 0x2e, 0x32, 0xe8, 0x0f, 0xa0, 0xeb, 0x24, 0x87, 0xb6, 0x30, 0x89, 0x90, 0xd9, 0xdb, 0x7a, 0xf7,
+ 0xd8, 0x26, 0xdd, 0xdc, 0xde, 0x7d, 0x28, 0xa0, 0x56, 0xc7, 0x49, 0x0e, 0xc5, 0x08, 0x7d, 0x0e,
+ 0xf0, 0x55, 0x42, 0x43, 0xc5, 0x59, 0x6e, 0xfc, 0x7b, 0xc7, 0xe7, 0xfc, 0xfd, 0xdd, 0xfb, 0x3b,
+ 0x92, 0x75, 0x97, 0xb3, 0x93, 0xbc, 0x1d, 0x58, 0x8c, 0x70, 0xfc, 0x78, 0x48, 0x98, 0x62, 0x2f,
+ 0x7d, 0xe1, 0x83, 0xe3, 0xb3, 0xff, 0x58, 0xb2, 0x91, 0x12, 0x16, 0xa2, 0xcc, 0x97, 0xf1, 0xed,
+ 0x34, 0x74, 0xb4, 0x5e, 0xbc, 0x36, 0x13, 0x1e, 0x2e, 0x5f, 0x28, 0x6c, 0x2f, 0x1c, 0x50, 0x65,
+ 0xd1, 0x53, 0x1c, 0x2e, 0x1f, 0x29, 0x44, 0x6e, 0xbf, 0x0c, 0xcb, 0x31, 0x71, 0x68, 0xec, 0xf2,
+ 0x1b, 0xac, 0x17, 0x78, 0xdc, 0xed, 0xe5, 0x5e, 0x2e, 0x49, 0xf8, 0x2d, 0x0d, 0x46, 0xaf, 0xc0,
+ 0x92, 0xd8, 0xf6, 0x0c, 0x66, 0x5b, 0xf3, 0x24, 0x7e, 0x06, 0xf1, 0x32, 0x2c, 0x3f, 0x1e, 0xf2,
+ 0xbc, 0xe1, 0xec, 0xe3, 0x18, 0x3b, 0x8c, 0xa6, 0x6f, 0x05, 0x4b, 0x02, 0xbe, 0x9d, 0x82, 0xd1,
+ 0x9b, 0xb0, 0x26, 0x51, 0x49, 0xe2, 0xe0, 0x28, 0xa5, 0x20, 0xb1, 0x2a, 0x25, 0x57, 0xc5, 0xec,
+ 0x6d, 0x31, 0xb9, 0xad, 0xe7, 0x90, 0x01, 0x1d, 0x87, 0x06, 0x01, 0x09, 0x59, 0x22, 0x0e, 0xb7,
+ 0xae, 0x95, 0x7e, 0xa3, 0x1b, 0x70, 0x0e, 0xfb, 0x3e, 0xfd, 0xda, 0x16, 0x94, 0xae, 0x5d, 0xd2,
+ 0x4e, 0x16, 0x96, 0x86, 0x40, 0xfa, 0x44, 0xe0, 0x58, 0x79, 0x45, 0x8d, 0x0b, 0xd0, 0x4d, 0xf7,
+ 0x91, 0xdf, 0x07, 0x32, 0x0e, 0x29, 0xc6, 0xc6, 0x29, 0x58, 0xc8, 0xee, 0x84, 0xf1, 0xb7, 0x36,
+ 0xac, 0x54, 0x04, 0x15, 0xfa, 0x02, 0x80, 0x7b, 0xab, 0x0c, 0x2d, 0xe5, 0xae, 0xff, 0x73, 0xfc,
+ 0xe0, 0xe4, 0xfe, 0x2a, 0xc1, 0x16, 0xf7, 0x7e, 0x39, 0x44, 0x3f, 0x82, 0x9e, 0xf0, 0x58, 0xc5,
+ 0x5d, 0xba, 0xec, 0xfb, 0xdf, 0x81, 0x3b, 0xd7, 0x55, 0xb1, 0x17, 0x31, 0x20, 0xc7, 0xc6, 0x5f,
+ 0x5a, 0xd0, 0x4d, 0x05, 0xf3, 0xdb, 0x8d, 0xdc, 0x28, 0xb1, 0xd7, 0x89, 0xbe, 0xdd, 0x08, 0xd8,
+ 0x1d, 0x01, 0xfa, 0x8f, 0x74, 0x25, 0xe3, 0x6d, 0x80, 0xb1, 0xfe, 0x95, 0x2a, 0xb4, 0x2a, 0x55,
+ 0x30, 0x2f, 0xc3, 0x22, 0xb7, 0xac, 0x47, 0xdc, 0x5d, 0x16, 0x7b, 0x91, 0x68, 0x93, 0x4a, 0x9c,
+ 0x44, 0x95, 0x87, 0xfa, 0x73, 0xeb, 0x0f, 0x06, 0x2c, 0x64, 0x9f, 0xc7, 0xd0, 0x97, 0xd0, 0xcb,
+ 0xb4, 0x83, 0xd1, 0x8b, 0xe5, 0x4d, 0x2b, 0xb7, 0x97, 0x8d, 0x97, 0x26, 0x60, 0xa9, 0x0a, 0x6a,
+ 0x0a, 0x85, 0x70, 0xba, 0xd4, 0x53, 0x45, 0x57, 0xca, 0xd4, 0x75, 0x1d, 0x5b, 0xe3, 0xd5, 0x46,
+ 0xb8, 0xa9, 0x3c, 0x06, 0x2b, 0x15, 0x4d, 0x52, 0xf4, 0xda, 0x04, 0x2e, 0xb9, 0x46, 0xad, 0x71,
+ 0xb5, 0x21, 0x76, 0x2a, 0xf5, 0x31, 0xa0, 0x72, 0x07, 0x15, 0xbd, 0x3a, 0x91, 0xcd, 0xb8, 0x43,
+ 0x6b, 0xbc, 0xd6, 0x0c, 0xb9, 0x56, 0x51, 0xd9, 0x5b, 0x9d, 0xa8, 0x68, 0xae, 0x7b, 0x3b, 0x51,
+ 0xd1, 0x42, 0xc3, 0x76, 0x0a, 0x1d, 0xc0, 0x72, 0xb1, 0xef, 0x8a, 0x2e, 0xd7, 0xfd, 0x27, 0x50,
+ 0x6a, 0xeb, 0x1a, 0x57, 0x9a, 0xa0, 0xa6, 0xc2, 0x08, 0x9c, 0xca, 0xf7, 0x39, 0xd1, 0x2b, 0x65,
+ 0xfa, 0xca, 0x4e, 0xaf, 0x71, 0x69, 0x32, 0x62, 0x56, 0xa7, 0x62, 0xef, 0xb3, 0x4a, 0xa7, 0x9a,
+ 0xc6, 0x6a, 0x95, 0x4e, 0x75, 0xad, 0x54, 0x73, 0x0a, 0xfd, 0x58, 0x37, 0xd4, 0x0a, 0x3d, 0x41,
+ 0xb4, 0x59, 0xc7, 0xa6, 0xba, 0x29, 0x69, 0x5c, 0x6b, 0x8c, 0xaf, 0x65, 0x5f, 0x6f, 0xf1, 0x58,
+ 0xcf, 0xb4, 0x06, 0xab, 0x62, 0xbd, 0xdc, 0x6c, 0xac, 0x8a, 0xf5, 0xaa, 0xfe, 0xe2, 0x14, 0xda,
+ 0x83, 0xc5, 0x5c, 0xb3, 0x10, 0xbd, 0x5c, 0x47, 0x99, 0x7f, 0xdd, 0x33, 0x5e, 0x99, 0x88, 0x97,
+ 0xca, 0xb0, 0x75, 0xf6, 0x52, 0xe9, 0xaa, 0x76, 0x71, 0xf9, 0x7c, 0xf5, 0xf2, 0x24, 0xb4, 0x5c,
+ 0x28, 0x97, 0x5a, 0x8a, 0x95, 0xa1, 0x5c, 0xd7, 0xb2, 0xac, 0x0c, 0xe5, 0xfa, 0x2e, 0xe5, 0x14,
+ 0xfa, 0xa1, 0x2e, 0x70, 0x85, 0x23, 0x5c, 0xac, 0xa3, 0xce, 0xee, 0xfe, 0x8b, 0x47, 0x23, 0xa5,
+ 0xac, 0xbf, 0x86, 0xd5, 0xaa, 0x57, 0x28, 0x74, 0xb5, 0xaa, 0x6c, 0xae, 0x7d, 0xea, 0x32, 0x36,
+ 0x9b, 0xa2, 0xa7, 0x82, 0x3f, 0x85, 0x8e, 0x6e, 0xb9, 0xa1, 0x17, 0xca, 0xd4, 0x85, 0x26, 0xa5,
+ 0x61, 0x1e, 0x85, 0x92, 0x71, 0xe0, 0x40, 0xc7, 0xea, 0xb8, 0x17, 0x56, 0x1f, 0xab, 0xa5, 0xae,
+ 0x5d, 0x7d, 0xac, 0x96, 0x5b, 0x6b, 0x42, 0x5c, 0xea, 0x0c, 0xd9, 0xd6, 0x51, 0xbd, 0x33, 0x54,
+ 0x74, 0xc6, 0xea, 0x9d, 0xa1, 0xb2, 0x1b, 0x35, 0x85, 0x7e, 0x02, 0x6b, 0xd5, 0x1d, 0x23, 0x54,
+ 0x1b, 0xf1, 0x35, 0x9d, 0x2b, 0xe3, 0x7a, 0x73, 0x82, 0x54, 0xfc, 0x53, 0x9d, 0x9f, 0x0a, 0x1d,
+ 0xa3, 0xfa, 0xfc, 0x54, 0xdd, 0xb7, 0x32, 0xae, 0x35, 0xc6, 0x2f, 0x87, 0x5e, 0xb6, 0xa5, 0x52,
+ 0x6f, 0xed, 0x8a, 0x2e, 0x54, 0xbd, 0xb5, 0x2b, 0xbb, 0x34, 0x22, 0x3e, 0xaa, 0xda, 0x25, 0x55,
+ 0xf1, 0x71, 0x44, 0x3f, 0xc7, 0xd8, 0x6c, 0x8a, 0x9e, 0x3b, 0xbe, 0xcb, 0xfd, 0x10, 0x34, 0x71,
+ 0xfd, 0xb9, 0xcc, 0x7c, 0xb5, 0x21, 0x76, 0xfd, 0xee, 0xea, 0x4c, 0x3d, 0x51, 0x81, 0x42, 0xc6,
+ 0xbe, 0xd6, 0x18, 0x3f, 0x95, 0x1d, 0xe9, 0x9f, 0x31, 0x32, 0xbd, 0x0c, 0x74, 0x65, 0x02, 0x9f,
+ 0x4c, 0x2f, 0xc6, 0x78, 0xb5, 0x11, 0x6e, 0x55, 0xf4, 0x66, 0xbb, 0x0b, 0x47, 0xf9, 0x53, 0xa9,
+ 0x25, 0x72, 0x94, 0x3f, 0x55, 0x34, 0x2c, 0x2a, 0xa2, 0x57, 0x37, 0x15, 0x26, 0x47, 0x6f, 0xa1,
+ 0xb9, 0x31, 0x39, 0x7a, 0x4b, 0xfd, 0x8a, 0x29, 0xf4, 0xb3, 0x71, 0x93, 0xbe, 0xfc, 0x08, 0x88,
+ 0xb6, 0x6a, 0x53, 0x51, 0xed, 0xdb, 0xa7, 0xf1, 0xc6, 0xb1, 0x68, 0x32, 0xc6, 0xff, 0x65, 0x4b,
+ 0x77, 0xfc, 0x2a, 0x5f, 0xe1, 0xd0, 0x9b, 0x0d, 0x18, 0x97, 0x1e, 0x12, 0x8d, 0xb7, 0x8e, 0x49,
+ 0x95, 0x59, 0xd0, 0x47, 0x30, 0x2b, 0xaa, 0x4f, 0x74, 0xfe, 0xe8, 0xb2, 0xd4, 0xb8, 0x50, 0x3d,
+ 0x9f, 0x16, 0x57, 0x9c, 0xdb, 0xde, 0x9c, 0xf8, 0x1d, 0xf7, 0x8d, 0x7f, 0x06, 0x00, 0x00, 0xff,
+ 0xff, 0x96, 0x31, 0x6f, 0x58, 0xa5, 0x2b, 0x00, 0x00,
}
diff --git a/weed/pb/volume_server_pb/volume_server_helper.go b/weed/pb/volume_server_pb/volume_server_helper.go
new file mode 100644
index 000000000..356be27ff
--- /dev/null
+++ b/weed/pb/volume_server_pb/volume_server_helper.go
@@ -0,0 +1,5 @@
+package volume_server_pb
+
+func (m *RemoteFile) BackendName() string {
+ return m.BackendType + "." + m.BackendId
+}
diff --git a/weed/query/json/query_json.go b/weed/query/json/query_json.go
new file mode 100644
index 000000000..46f3b1b56
--- /dev/null
+++ b/weed/query/json/query_json.go
@@ -0,0 +1,107 @@
+package json
+
+import (
+ "strconv"
+
+ "github.com/chrislusf/seaweedfs/weed/query/sqltypes"
+ "github.com/tidwall/gjson"
+ "github.com/tidwall/match"
+)
+
+type Query struct {
+ Field string
+ Op string
+ Value string
+}
+
+func QueryJson(jsonLine string, projections []string, query Query) (passedFilter bool, values []sqltypes.Value) {
+ if filterJson(jsonLine, query) {
+ passedFilter = true
+ fields := gjson.GetMany(jsonLine, projections...)
+ for _, f := range fields {
+ values = append(values, sqltypes.MakeTrusted(sqltypes.Type(f.Type), sqltypes.StringToBytes(f.Raw)))
+ }
+ return
+ }
+ return false, nil
+}
+
+func filterJson(jsonLine string, query Query) bool {
+
+ value := gjson.Get(jsonLine, query.Field)
+
+ // copied from gjson.go queryMatches() function
+ rpv := query.Value
+
+ if !value.Exists() {
+ return false
+ }
+ if query.Op == "" {
+ // the query is only looking for existence, such as:
+ // friends.#(name)
+ // which makes sure that the array "friends" has an element of
+ // "name" that exists
+ return true
+ }
+ switch value.Type {
+ case gjson.String:
+ switch query.Op {
+ case "=":
+ return value.Str == rpv
+ case "!=":
+ return value.Str != rpv
+ case "<":
+ return value.Str < rpv
+ case "<=":
+ return value.Str <= rpv
+ case ">":
+ return value.Str > rpv
+ case ">=":
+ return value.Str >= rpv
+ case "%":
+ return match.Match(value.Str, rpv)
+ case "!%":
+ return !match.Match(value.Str, rpv)
+ }
+ case gjson.Number:
+ rpvn, _ := strconv.ParseFloat(rpv, 64)
+ switch query.Op {
+ case "=":
+ return value.Num == rpvn
+ case "!=":
+ return value.Num != rpvn
+ case "<":
+ return value.Num < rpvn
+ case "<=":
+ return value.Num <= rpvn
+ case ">":
+ return value.Num > rpvn
+ case ">=":
+ return value.Num >= rpvn
+ }
+ case gjson.True:
+ switch query.Op {
+ case "=":
+ return rpv == "true"
+ case "!=":
+ return rpv != "true"
+ case ">":
+ return rpv == "false"
+ case ">=":
+ return true
+ }
+ case gjson.False:
+ switch query.Op {
+ case "=":
+ return rpv == "false"
+ case "!=":
+ return rpv != "false"
+ case "<":
+ return rpv == "true"
+ case "<=":
+ return true
+ }
+ }
+ return false
+
+}
diff --git a/weed/query/json/query_json_test.go b/weed/query/json/query_json_test.go
new file mode 100644
index 000000000..1794bb333
--- /dev/null
+++ b/weed/query/json/query_json_test.go
@@ -0,0 +1,133 @@
+package json
+
+import (
+ "testing"
+
+ "github.com/tidwall/gjson"
+)
+
+func TestGjson(t *testing.T) {
+ data := `
+ {
+ "quiz": {
+ "sport": {
+ "q1": {
+ "question": "Which one is correct team name in NBA?",
+ "options": [
+ "New York Bulls",
+ "Los Angeles Kings",
+ "Golden State Warriros",
+ "Huston Rocket"
+ ],
+ "answer": "Huston Rocket"
+ }
+ },
+ "maths": {
+ "q1": {
+ "question": "5 + 7 = ?",
+ "options": [
+ "10",
+ "11",
+ "12",
+ "13"
+ ],
+ "answer": "12"
+ },
+ "q2": {
+ "question": "12 - 8 = ?",
+ "options": [
+ "1",
+ "2",
+ "3",
+ "4"
+ ],
+ "answer": "4"
+ }
+ }
+ }
+ }
+
+ {
+ "fruit": "Apple",
+ "size": "Large",
+ "quiz": "Red"
+ }
+
+`
+
+ projections := []string{"quiz", "fruit"}
+
+ gjson.ForEachLine(data, func(line gjson.Result) bool {
+ println(line.Raw)
+ println("+++++++++++")
+ results := gjson.GetMany(line.Raw, projections...)
+ for _, result := range results {
+ println(result.Index, result.Type, result.String())
+ }
+ println("-----------")
+ return true
+ })
+
+}
+
+func TestJsonQueryRow(t *testing.T) {
+
+ data := `
+ {
+ "fruit": "Bl\"ue",
+ "size": 6,
+ "quiz": "green"
+ }
+
+`
+ selections := []string{"fruit", "size"}
+
+ isFiltered, values := QueryJson(data, selections, Query{
+ Field: "quiz",
+ Op: "=",
+ Value: "green",
+ })
+
+ if !isFiltered {
+ t.Errorf("should have been filtered")
+ }
+
+ if values == nil {
+ t.Errorf("values should have been returned")
+ }
+
+ buf := ToJson(nil, selections, values)
+ println(string(buf))
+
+}
+
+func TestJsonQueryNumber(t *testing.T) {
+
+ data := `
+ {
+ "fruit": "Bl\"ue",
+ "size": 6,
+ "quiz": "green"
+ }
+
+`
+ selections := []string{"fruit", "quiz"}
+
+ isFiltered, values := QueryJson(data, selections, Query{
+ Field: "size",
+ Op: ">=",
+ Value: "6",
+ })
+
+ if !isFiltered {
+ t.Errorf("should have been filtered")
+ }
+
+ if values == nil {
+ t.Errorf("values should have been returned")
+ }
+
+ buf := ToJson(nil, selections, values)
+ println(string(buf))
+
+}
diff --git a/weed/query/json/seralize.go b/weed/query/json/seralize.go
new file mode 100644
index 000000000..9bbddc2ff
--- /dev/null
+++ b/weed/query/json/seralize.go
@@ -0,0 +1,17 @@
+package json
+
+import "github.com/chrislusf/seaweedfs/weed/query/sqltypes"
+
+func ToJson(buf []byte, selections []string, values []sqltypes.Value) []byte {
+ buf = append(buf, '{')
+ for i, value := range values {
+ if i > 0 {
+ buf = append(buf, ',')
+ }
+ buf = append(buf, selections[i]...)
+ buf = append(buf, ':')
+ buf = append(buf, value.Raw()...)
+ }
+ buf = append(buf, '}')
+ return buf
+}
diff --git a/weed/query/sqltypes/const.go b/weed/query/sqltypes/const.go
new file mode 100644
index 000000000..f1d0f540e
--- /dev/null
+++ b/weed/query/sqltypes/const.go
@@ -0,0 +1,124 @@
+package sqltypes
+
+// copied from vitness
+
+// Flag allows us to qualify types by their common properties.
+type Flag int32
+
+const (
+ Flag_NONE Flag = 0
+ Flag_ISINTEGRAL Flag = 256
+ Flag_ISUNSIGNED Flag = 512
+ Flag_ISFLOAT Flag = 1024
+ Flag_ISQUOTED Flag = 2048
+ Flag_ISTEXT Flag = 4096
+ Flag_ISBINARY Flag = 8192
+)
+
+var Flag_name = map[int32]string{
+ 0: "NONE",
+ 256: "ISINTEGRAL",
+ 512: "ISUNSIGNED",
+ 1024: "ISFLOAT",
+ 2048: "ISQUOTED",
+ 4096: "ISTEXT",
+ 8192: "ISBINARY",
+}
+var Flag_value = map[string]int32{
+ "NONE": 0,
+ "ISINTEGRAL": 256,
+ "ISUNSIGNED": 512,
+ "ISFLOAT": 1024,
+ "ISQUOTED": 2048,
+ "ISTEXT": 4096,
+ "ISBINARY": 8192,
+}
+
+// Type defines the various supported data types in bind vars
+// and query results.
+type Type int32
+
+const (
+ // NULL_TYPE specifies a NULL type.
+ Type_NULL_TYPE Type = 0
+ // INT8 specifies a TINYINT type.
+ // Properties: 1, IsNumber.
+ Type_INT8 Type = 257
+ // UINT8 specifies a TINYINT UNSIGNED type.
+ // Properties: 2, IsNumber, IsUnsigned.
+ Type_UINT8 Type = 770
+ // INT16 specifies a SMALLINT type.
+ // Properties: 3, IsNumber.
+ Type_INT16 Type = 259
+ // UINT16 specifies a SMALLINT UNSIGNED type.
+ // Properties: 4, IsNumber, IsUnsigned.
+ Type_UINT16 Type = 772
+ // INT24 specifies a MEDIUMINT type.
+ // Properties: 5, IsNumber.
+ Type_INT32 Type = 263
+ // UINT32 specifies a INTEGER UNSIGNED type.
+ // Properties: 8, IsNumber, IsUnsigned.
+ Type_UINT32 Type = 776
+ // INT64 specifies a BIGINT type.
+ // Properties: 9, IsNumber.
+ Type_INT64 Type = 265
+ // UINT64 specifies a BIGINT UNSIGNED type.
+ // Properties: 10, IsNumber, IsUnsigned.
+ Type_UINT64 Type = 778
+ // FLOAT32 specifies a FLOAT type.
+ // Properties: 11, IsFloat.
+ Type_FLOAT32 Type = 1035
+ // FLOAT64 specifies a DOUBLE or REAL type.
+ // Properties: 12, IsFloat.
+ Type_FLOAT64 Type = 1036
+ // TIMESTAMP specifies a TIMESTAMP type.
+ // Properties: 13, IsQuoted.
+ Type_TIMESTAMP Type = 2061
+ // DATE specifies a DATE type.
+ // Properties: 14, IsQuoted.
+ Type_DATE Type = 2062
+ // TIME specifies a TIME type.
+ // Properties: 15, IsQuoted.
+ Type_TIME Type = 2063
+ // DATETIME specifies a DATETIME type.
+ // Properties: 16, IsQuoted.
+ Type_DATETIME Type = 2064
+ // YEAR specifies a YEAR type.
+ // Properties: 17, IsNumber, IsUnsigned.
+ Type_YEAR Type = 785
+ // DECIMAL specifies a DECIMAL or NUMERIC type.
+ // Properties: 18, None.
+ Type_DECIMAL Type = 18
+ // TEXT specifies a TEXT type.
+ // Properties: 19, IsQuoted, IsText.
+ Type_TEXT Type = 6163
+ // BLOB specifies a BLOB type.
+ // Properties: 20, IsQuoted, IsBinary.
+ Type_BLOB Type = 10260
+ // VARCHAR specifies a VARCHAR type.
+ // Properties: 21, IsQuoted, IsText.
+ Type_VARCHAR Type = 6165
+ // VARBINARY specifies a VARBINARY type.
+ // Properties: 22, IsQuoted, IsBinary.
+ Type_VARBINARY Type = 10262
+ // CHAR specifies a CHAR type.
+ // Properties: 23, IsQuoted, IsText.
+ Type_CHAR Type = 6167
+ // BINARY specifies a BINARY type.
+ // Properties: 24, IsQuoted, IsBinary.
+ Type_BINARY Type = 10264
+ // BIT specifies a BIT type.
+ // Properties: 25, IsQuoted.
+ Type_BIT Type = 2073
+ // JSON specifies a JSON type.
+ // Properties: 30, IsQuoted.
+ Type_JSON Type = 2078
+)
+
+// BindVariable represents a single bind variable in a Query.
+type BindVariable struct {
+ Type Type `protobuf:"varint,1,opt,name=type,enum=query.Type" json:"type,omitempty"`
+ Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+ // values are set if type is TUPLE.
+ Values []*Value `protobuf:"bytes,3,rep,name=values" json:"values,omitempty"`
+}
diff --git a/weed/query/sqltypes/type.go b/weed/query/sqltypes/type.go
new file mode 100644
index 000000000..f4f3dd471
--- /dev/null
+++ b/weed/query/sqltypes/type.go
@@ -0,0 +1,101 @@
+package sqltypes
+
+// These bit flags can be used to query on the
+// common properties of types.
+const (
+ flagIsIntegral = int(Flag_ISINTEGRAL)
+ flagIsUnsigned = int(Flag_ISUNSIGNED)
+ flagIsFloat = int(Flag_ISFLOAT)
+ flagIsQuoted = int(Flag_ISQUOTED)
+ flagIsText = int(Flag_ISTEXT)
+ flagIsBinary = int(Flag_ISBINARY)
+)
+
+// IsIntegral returns true if Type is an integral
+// (signed/unsigned) that can be represented using
+// up to 64 binary bits.
+// If you have a Value object, use its member function.
+func IsIntegral(t Type) bool {
+ return int(t)&flagIsIntegral == flagIsIntegral
+}
+
+// IsSigned returns true if Type is a signed integral.
+// If you have a Value object, use its member function.
+func IsSigned(t Type) bool {
+ return int(t)&(flagIsIntegral|flagIsUnsigned) == flagIsIntegral
+}
+
+// IsUnsigned returns true if Type is an unsigned integral.
+// Caution: this is not the same as !IsSigned.
+// If you have a Value object, use its member function.
+func IsUnsigned(t Type) bool {
+ return int(t)&(flagIsIntegral|flagIsUnsigned) == flagIsIntegral|flagIsUnsigned
+}
+
+// IsFloat returns true is Type is a floating point.
+// If you have a Value object, use its member function.
+func IsFloat(t Type) bool {
+ return int(t)&flagIsFloat == flagIsFloat
+}
+
+// IsQuoted returns true if Type is a quoted text or binary.
+// If you have a Value object, use its member function.
+func IsQuoted(t Type) bool {
+ return (int(t)&flagIsQuoted == flagIsQuoted) && t != Bit
+}
+
+// IsText returns true if Type is a text.
+// If you have a Value object, use its member function.
+func IsText(t Type) bool {
+ return int(t)&flagIsText == flagIsText
+}
+
+// IsBinary returns true if Type is a binary.
+// If you have a Value object, use its member function.
+func IsBinary(t Type) bool {
+ return int(t)&flagIsBinary == flagIsBinary
+}
+
+// isNumber returns true if the type is any type of number.
+func isNumber(t Type) bool {
+ return IsIntegral(t) || IsFloat(t) || t == Decimal
+}
+
+// IsTemporal returns true if Value is time type.
+func IsTemporal(t Type) bool {
+ switch t {
+ case Timestamp, Date, Time, Datetime:
+ return true
+ }
+ return false
+}
+
+// Vitess data types. These are idiomatically
+// named synonyms for the Type values.
+const (
+ Null = Type_NULL_TYPE
+ Int8 = Type_INT8
+ Uint8 = Type_UINT8
+ Int16 = Type_INT16
+ Uint16 = Type_UINT16
+ Int32 = Type_INT32
+ Uint32 = Type_UINT32
+ Int64 = Type_INT64
+ Uint64 = Type_UINT64
+ Float32 = Type_FLOAT32
+ Float64 = Type_FLOAT64
+ Timestamp = Type_TIMESTAMP
+ Date = Type_DATE
+ Time = Type_TIME
+ Datetime = Type_DATETIME
+ Year = Type_YEAR
+ Decimal = Type_DECIMAL
+ Text = Type_TEXT
+ Blob = Type_BLOB
+ VarChar = Type_VARCHAR
+ VarBinary = Type_VARBINARY
+ Char = Type_CHAR
+ Binary = Type_BINARY
+ Bit = Type_BIT
+ TypeJSON = Type_JSON
+)
diff --git a/weed/query/sqltypes/unsafe.go b/weed/query/sqltypes/unsafe.go
new file mode 100644
index 000000000..e322c92ce
--- /dev/null
+++ b/weed/query/sqltypes/unsafe.go
@@ -0,0 +1,30 @@
+package sqltypes
+
+import (
+ "reflect"
+ "unsafe"
+)
+
+// BytesToString casts slice to string without copy
+func BytesToString(b []byte) (s string) {
+ if len(b) == 0 {
+ return ""
+ }
+
+ bh := (*reflect.SliceHeader)(unsafe.Pointer(&b))
+ sh := reflect.StringHeader{Data: bh.Data, Len: bh.Len}
+
+ return *(*string)(unsafe.Pointer(&sh))
+}
+
+// StringToBytes casts string to slice without copy
+func StringToBytes(s string) []byte {
+ if len(s) == 0 {
+ return []byte{}
+ }
+
+ sh := (*reflect.StringHeader)(unsafe.Pointer(&s))
+ bh := reflect.SliceHeader{Data: sh.Data, Len: sh.Len, Cap: sh.Len}
+
+ return *(*[]byte)(unsafe.Pointer(&bh))
+}
diff --git a/weed/query/sqltypes/value.go b/weed/query/sqltypes/value.go
new file mode 100644
index 000000000..012de2b45
--- /dev/null
+++ b/weed/query/sqltypes/value.go
@@ -0,0 +1,355 @@
+package sqltypes
+
+import (
+ "fmt"
+ "strconv"
+ "time"
+)
+
+var (
+ // NULL represents the NULL value.
+ NULL = Value{}
+ // DontEscape tells you if a character should not be escaped.
+ DontEscape = byte(255)
+ nullstr = []byte("null")
+)
+
+type Value struct {
+ typ Type
+ val []byte
+}
+
+// NewValue builds a Value using typ and val. If the value and typ
+// don't match, it returns an error.
+func NewValue(typ Type, val []byte) (v Value, err error) {
+ switch {
+ case IsSigned(typ):
+ if _, err := strconv.ParseInt(string(val), 0, 64); err != nil {
+ return NULL, err
+ }
+ return MakeTrusted(typ, val), nil
+ case IsUnsigned(typ):
+ if _, err := strconv.ParseUint(string(val), 0, 64); err != nil {
+ return NULL, err
+ }
+ return MakeTrusted(typ, val), nil
+ case IsFloat(typ) || typ == Decimal:
+ if _, err := strconv.ParseFloat(string(val), 64); err != nil {
+ return NULL, err
+ }
+ return MakeTrusted(typ, val), nil
+ case IsQuoted(typ) || typ == Bit || typ == Null:
+ return MakeTrusted(typ, val), nil
+ }
+ // All other types are unsafe or invalid.
+ return NULL, fmt.Errorf("invalid type specified for MakeValue: %v", typ)
+}
+
+// MakeTrusted makes a new Value based on the type.
+// This function should only be used if you know the value
+// and type conform to the rules. Every place this function is
+// called, a comment is needed that explains why it's justified.
+// Exceptions: The current package and mysql package do not need
+// comments. Other packages can also use the function to create
+// VarBinary or VarChar values.
+func MakeTrusted(typ Type, val []byte) Value {
+
+ if typ == Null {
+ return NULL
+ }
+
+ return Value{typ: typ, val: val}
+}
+
+// NewInt64 builds an Int64 Value.
+func NewInt64(v int64) Value {
+ return MakeTrusted(Int64, strconv.AppendInt(nil, v, 10))
+}
+
+// NewInt32 builds an Int64 Value.
+func NewInt32(v int32) Value {
+ return MakeTrusted(Int32, strconv.AppendInt(nil, int64(v), 10))
+}
+
+// NewUint64 builds an Uint64 Value.
+func NewUint64(v uint64) Value {
+ return MakeTrusted(Uint64, strconv.AppendUint(nil, v, 10))
+}
+
+// NewFloat32 builds an Float64 Value.
+func NewFloat32(v float32) Value {
+ return MakeTrusted(Float32, strconv.AppendFloat(nil, float64(v), 'f', -1, 64))
+}
+
+// NewFloat64 builds an Float64 Value.
+func NewFloat64(v float64) Value {
+ return MakeTrusted(Float64, strconv.AppendFloat(nil, v, 'g', -1, 64))
+}
+
+// NewVarChar builds a VarChar Value.
+func NewVarChar(v string) Value {
+ return MakeTrusted(VarChar, []byte(v))
+}
+
+// NewVarBinary builds a VarBinary Value.
+// The input is a string because it's the most common use case.
+func NewVarBinary(v string) Value {
+ return MakeTrusted(VarBinary, []byte(v))
+}
+
+// NewIntegral builds an integral type from a string representation.
+// The type will be Int64 or Uint64. Int64 will be preferred where possible.
+func NewIntegral(val string) (n Value, err error) {
+ signed, err := strconv.ParseInt(val, 0, 64)
+ if err == nil {
+ return MakeTrusted(Int64, strconv.AppendInt(nil, signed, 10)), nil
+ }
+ unsigned, err := strconv.ParseUint(val, 0, 64)
+ if err != nil {
+ return Value{}, err
+ }
+ return MakeTrusted(Uint64, strconv.AppendUint(nil, unsigned, 10)), nil
+}
+
+// MakeString makes a VarBinary Value.
+func MakeString(val []byte) Value {
+ return MakeTrusted(VarBinary, val)
+}
+
+// BuildValue builds a value from any go type. sqltype.Value is
+// also allowed.
+func BuildValue(goval interface{}) (v Value, err error) {
+ // Look for the most common types first.
+ switch goval := goval.(type) {
+ case nil:
+ // no op
+ case []byte:
+ v = MakeTrusted(VarBinary, goval)
+ case int64:
+ v = MakeTrusted(Int64, strconv.AppendInt(nil, int64(goval), 10))
+ case uint64:
+ v = MakeTrusted(Uint64, strconv.AppendUint(nil, uint64(goval), 10))
+ case float64:
+ v = MakeTrusted(Float64, strconv.AppendFloat(nil, goval, 'f', -1, 64))
+ case int:
+ v = MakeTrusted(Int64, strconv.AppendInt(nil, int64(goval), 10))
+ case int8:
+ v = MakeTrusted(Int8, strconv.AppendInt(nil, int64(goval), 10))
+ case int16:
+ v = MakeTrusted(Int16, strconv.AppendInt(nil, int64(goval), 10))
+ case int32:
+ v = MakeTrusted(Int32, strconv.AppendInt(nil, int64(goval), 10))
+ case uint:
+ v = MakeTrusted(Uint64, strconv.AppendUint(nil, uint64(goval), 10))
+ case uint8:
+ v = MakeTrusted(Uint8, strconv.AppendUint(nil, uint64(goval), 10))
+ case uint16:
+ v = MakeTrusted(Uint16, strconv.AppendUint(nil, uint64(goval), 10))
+ case uint32:
+ v = MakeTrusted(Uint32, strconv.AppendUint(nil, uint64(goval), 10))
+ case float32:
+ v = MakeTrusted(Float32, strconv.AppendFloat(nil, float64(goval), 'f', -1, 64))
+ case string:
+ v = MakeTrusted(VarBinary, []byte(goval))
+ case time.Time:
+ v = MakeTrusted(Datetime, []byte(goval.Format("2006-01-02 15:04:05")))
+ case Value:
+ v = goval
+ case *BindVariable:
+ return ValueFromBytes(goval.Type, goval.Value)
+ default:
+ return v, fmt.Errorf("unexpected type %T: %v", goval, goval)
+ }
+ return v, nil
+}
+
+// BuildConverted is like BuildValue except that it tries to
+// convert a string or []byte to an integral if the target type
+// is an integral. We don't perform other implicit conversions
+// because they're unsafe.
+func BuildConverted(typ Type, goval interface{}) (v Value, err error) {
+ if IsIntegral(typ) {
+ switch goval := goval.(type) {
+ case []byte:
+ return ValueFromBytes(typ, goval)
+ case string:
+ return ValueFromBytes(typ, []byte(goval))
+ case Value:
+ if goval.IsQuoted() {
+ return ValueFromBytes(typ, goval.Raw())
+ }
+ }
+ }
+ return BuildValue(goval)
+}
+
+// ValueFromBytes builds a Value using typ and val. It ensures that val
+// matches the requested type. If type is an integral it's converted to
+// a canonical form. Otherwise, the original representation is preserved.
+func ValueFromBytes(typ Type, val []byte) (v Value, err error) {
+ switch {
+ case IsSigned(typ):
+ signed, err := strconv.ParseInt(string(val), 0, 64)
+ if err != nil {
+ return NULL, err
+ }
+ v = MakeTrusted(typ, strconv.AppendInt(nil, signed, 10))
+ case IsUnsigned(typ):
+ unsigned, err := strconv.ParseUint(string(val), 0, 64)
+ if err != nil {
+ return NULL, err
+ }
+ v = MakeTrusted(typ, strconv.AppendUint(nil, unsigned, 10))
+ case IsFloat(typ) || typ == Decimal:
+ _, err := strconv.ParseFloat(string(val), 64)
+ if err != nil {
+ return NULL, err
+ }
+ // After verification, we preserve the original representation.
+ fallthrough
+ default:
+ v = MakeTrusted(typ, val)
+ }
+ return v, nil
+}
+
+// BuildIntegral builds an integral type from a string representation.
+// The type will be Int64 or Uint64. Int64 will be preferred where possible.
+func BuildIntegral(val string) (n Value, err error) {
+ signed, err := strconv.ParseInt(val, 0, 64)
+ if err == nil {
+ return MakeTrusted(Int64, strconv.AppendInt(nil, signed, 10)), nil
+ }
+ unsigned, err := strconv.ParseUint(val, 0, 64)
+ if err != nil {
+ return Value{}, err
+ }
+ return MakeTrusted(Uint64, strconv.AppendUint(nil, unsigned, 10)), nil
+}
+
+// Type returns the type of Value.
+func (v Value) Type() Type {
+ return v.typ
+}
+
+// Raw returns the raw bytes. All types are currently implemented as []byte.
+// You should avoid using this function. If you do, you should treat the
+// bytes as read-only.
+func (v Value) Raw() []byte {
+ return v.val
+}
+
+// Len returns the length.
+func (v Value) Len() int {
+ return len(v.val)
+}
+
+// Values represents the array of Value.
+type Values []Value
+
+// Len implements the interface.
+func (vs Values) Len() int {
+ len := 0
+ for _, v := range vs {
+ len += v.Len()
+ }
+ return len
+}
+
+// String returns the raw value as a string.
+func (v Value) String() string {
+ return BytesToString(v.val)
+}
+
+// ToNative converts Value to a native go type.
+// This does not work for sqltypes.Tuple. The function
+// panics if there are inconsistencies.
+func (v Value) ToNative() interface{} {
+ var out interface{}
+ var err error
+ switch {
+ case v.typ == Null:
+ // no-op
+ case IsSigned(v.typ):
+ out, err = v.ParseInt64()
+ case IsUnsigned(v.typ):
+ out, err = v.ParseUint64()
+ case IsFloat(v.typ):
+ out, err = v.ParseFloat64()
+ default:
+ out = v.val
+ }
+ if err != nil {
+ panic(err)
+ }
+ return out
+}
+
+// ParseInt64 will parse a Value into an int64. It does
+// not check the type.
+func (v Value) ParseInt64() (val int64, err error) {
+ return strconv.ParseInt(v.String(), 10, 64)
+}
+
+// ParseUint64 will parse a Value into a uint64. It does
+// not check the type.
+func (v Value) ParseUint64() (val uint64, err error) {
+ return strconv.ParseUint(v.String(), 10, 64)
+}
+
+// ParseFloat64 will parse a Value into an float64. It does
+// not check the type.
+func (v Value) ParseFloat64() (val float64, err error) {
+ return strconv.ParseFloat(v.String(), 64)
+}
+
+// IsNull returns true if Value is null.
+func (v Value) IsNull() bool {
+ return v.typ == Null
+}
+
+// IsIntegral returns true if Value is an integral.
+func (v Value) IsIntegral() bool {
+ return IsIntegral(v.typ)
+}
+
+// IsSigned returns true if Value is a signed integral.
+func (v Value) IsSigned() bool {
+ return IsSigned(v.typ)
+}
+
+// IsUnsigned returns true if Value is an unsigned integral.
+func (v Value) IsUnsigned() bool {
+ return IsUnsigned(v.typ)
+}
+
+// IsFloat returns true if Value is a float.
+func (v Value) IsFloat() bool {
+ return IsFloat(v.typ)
+}
+
+// IsQuoted returns true if Value must be SQL-quoted.
+func (v Value) IsQuoted() bool {
+ return IsQuoted(v.typ)
+}
+
+// IsText returns true if Value is a collatable text.
+func (v Value) IsText() bool {
+ return IsText(v.typ)
+}
+
+// IsBinary returns true if Value is binary.
+func (v Value) IsBinary() bool {
+ return IsBinary(v.typ)
+}
+
+// IsTemporal returns true if Value is time type.
+func (v Value) IsTemporal() bool {
+ return IsTemporal(v.typ)
+}
+
+// ToString returns the value as MySQL would return it as string.
+// If the value is not convertible like in the case of Expression, it returns nil.
+func (v Value) ToString() string {
+ return BytesToString(v.val)
+}
diff --git a/weed/replication/sink/s3sink/s3_sink.go b/weed/replication/sink/s3sink/s3_sink.go
index d5cad3541..4cff341d0 100644
--- a/weed/replication/sink/s3sink/s3_sink.go
+++ b/weed/replication/sink/s3sink/s3_sink.go
@@ -56,7 +56,7 @@ func (s3sink *S3Sink) SetSourceFiler(s *source.FilerSource) {
s3sink.filerSource = s
}
-func (s3sink *S3Sink) initialize(awsAccessKeyId, aswSecretAccessKey, region, bucket, dir string) error {
+func (s3sink *S3Sink) initialize(awsAccessKeyId, awsSecretAccessKey, region, bucket, dir string) error {
s3sink.region = region
s3sink.bucket = bucket
s3sink.dir = dir
@@ -64,8 +64,8 @@ func (s3sink *S3Sink) initialize(awsAccessKeyId, aswSecretAccessKey, region, buc
config := &aws.Config{
Region: aws.String(s3sink.region),
}
- if awsAccessKeyId != "" && aswSecretAccessKey != "" {
- config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, aswSecretAccessKey, "")
+ if awsAccessKeyId != "" && awsSecretAccessKey != "" {
+ config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, awsSecretAccessKey, "")
}
sess, err := session.NewSession(config)
diff --git a/weed/replication/sub/notification_aws_sqs.go b/weed/replication/sub/notification_aws_sqs.go
index f0100f4de..bed26c79c 100644
--- a/weed/replication/sub/notification_aws_sqs.go
+++ b/weed/replication/sub/notification_aws_sqs.go
@@ -38,13 +38,13 @@ func (k *AwsSqsInput) Initialize(configuration util.Configuration) error {
)
}
-func (k *AwsSqsInput) initialize(awsAccessKeyId, aswSecretAccessKey, region, queueName string) (err error) {
+func (k *AwsSqsInput) initialize(awsAccessKeyId, awsSecretAccessKey, region, queueName string) (err error) {
config := &aws.Config{
Region: aws.String(region),
}
- if awsAccessKeyId != "" && aswSecretAccessKey != "" {
- config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, aswSecretAccessKey, "")
+ if awsAccessKeyId != "" && awsSecretAccessKey != "" {
+ config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, awsSecretAccessKey, "")
}
sess, err := session.NewSession(config)
diff --git a/weed/s3api/filer_multipart.go b/weed/s3api/filer_multipart.go
index 4de1dda36..d3bde66ee 100644
--- a/weed/s3api/filer_multipart.go
+++ b/weed/s3api/filer_multipart.go
@@ -14,7 +14,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
- "github.com/satori/go.uuid"
+ "github.com/google/uuid"
)
type InitiateMultipartUploadResult struct {
@@ -23,7 +23,7 @@ type InitiateMultipartUploadResult struct {
}
func (s3a *S3ApiServer) createMultipartUpload(ctx context.Context, input *s3.CreateMultipartUploadInput) (output *InitiateMultipartUploadResult, code ErrorCode) {
- uploadId, _ := uuid.NewV4()
+ uploadId, _ := uuid.NewRandom()
uploadIdString := uploadId.String()
if err := s3a.mkdir(ctx, s3a.genUploadsFolder(*input.Bucket), uploadIdString, func(entry *filer_pb.Entry) {
@@ -91,6 +91,11 @@ func (s3a *S3ApiServer) completeMultipartUpload(ctx context.Context, input *s3.C
}
dirName = fmt.Sprintf("%s/%s/%s", s3a.option.BucketsPath, *input.Bucket, dirName)
+ // remove suffix '/'
+ if strings.HasSuffix(dirName, "/") {
+ dirName = dirName[:len(dirName)-1]
+ }
+
err = s3a.mkFile(ctx, dirName, entryName, finalParts)
if err != nil {
@@ -101,9 +106,9 @@ func (s3a *S3ApiServer) completeMultipartUpload(ctx context.Context, input *s3.C
output = &CompleteMultipartUploadResult{
CompleteMultipartUploadOutput: s3.CompleteMultipartUploadOutput{
Location: aws.String(fmt.Sprintf("http://%s%s/%s", s3a.option.Filer, dirName, entryName)),
- Bucket: input.Bucket,
- ETag: aws.String("\"" + filer2.ETag(finalParts) + "\""),
- Key: objectKey(input.Key),
+ Bucket: input.Bucket,
+ ETag: aws.String("\"" + filer2.ETag(finalParts) + "\""),
+ Key: objectKey(input.Key),
},
}
diff --git a/weed/s3api/filer_util.go b/weed/s3api/filer_util.go
index 84e3050cd..ed9612d35 100644
--- a/weed/s3api/filer_util.go
+++ b/weed/s3api/filer_util.go
@@ -3,6 +3,7 @@ package s3api
import (
"context"
"fmt"
+ "io"
"os"
"strings"
"time"
@@ -37,6 +38,7 @@ func (s3a *S3ApiServer) mkdir(ctx context.Context, parentDirectoryPath string, d
glog.V(1).Infof("mkdir: %v", request)
if _, err := client.CreateEntry(ctx, request); err != nil {
+ glog.V(0).Infof("mkdir %v: %v", request, err)
return fmt.Errorf("mkdir %s/%s: %v", parentDirectoryPath, dirName, err)
}
@@ -67,6 +69,7 @@ func (s3a *S3ApiServer) mkFile(ctx context.Context, parentDirectoryPath string,
glog.V(1).Infof("create file: %s/%s", parentDirectoryPath, fileName)
if _, err := client.CreateEntry(ctx, request); err != nil {
+ glog.V(0).Infof("create file %v:%v", request, err)
return fmt.Errorf("create file %s/%s: %v", parentDirectoryPath, fileName, err)
}
@@ -87,12 +90,25 @@ func (s3a *S3ApiServer) list(ctx context.Context, parentDirectoryPath, prefix, s
}
glog.V(4).Infof("read directory: %v", request)
- resp, err := client.ListEntries(ctx, request)
+ stream, err := client.ListEntries(ctx, request)
if err != nil {
+ glog.V(0).Infof("read directory %v: %v", request, err)
return fmt.Errorf("list dir %v: %v", parentDirectoryPath, err)
}
- entries = resp.Entries
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ break
+ } else {
+ return recvErr
+ }
+ }
+
+ entries = append(entries, resp.Entry)
+
+ }
return nil
})
@@ -114,6 +130,7 @@ func (s3a *S3ApiServer) rm(ctx context.Context, parentDirectoryPath string, entr
glog.V(1).Infof("delete entry %v/%v: %v", parentDirectoryPath, entryName, request)
if _, err := client.DeleteEntry(ctx, request); err != nil {
+ glog.V(0).Infof("delete entry %v: %v", request, err)
return fmt.Errorf("delete entry %s/%s: %v", parentDirectoryPath, entryName, err)
}
@@ -134,6 +151,7 @@ func (s3a *S3ApiServer) exists(ctx context.Context, parentDirectoryPath string,
glog.V(4).Infof("exists entry %v/%v: %v", parentDirectoryPath, entryName, request)
resp, err := client.LookupDirectoryEntry(ctx, request)
if err != nil {
+ glog.V(0).Infof("exists entry %v: %v", request, err)
return fmt.Errorf("exists entry %s/%s: %v", parentDirectoryPath, entryName, err)
}
@@ -151,4 +169,4 @@ func objectKey(key *string) *string {
return &t
}
return key
-}
\ No newline at end of file
+}
diff --git a/weed/s3api/s3api_objects_list_handlers.go b/weed/s3api/s3api_objects_list_handlers.go
index 4053913fb..aa6849cbd 100644
--- a/weed/s3api/s3api_objects_list_handlers.go
+++ b/weed/s3api/s3api_objects_list_handlers.go
@@ -3,6 +3,7 @@ package s3api
import (
"context"
"fmt"
+ "io"
"net/http"
"net/url"
"path/filepath"
@@ -107,7 +108,7 @@ func (s3a *S3ApiServer) listFilerEntries(ctx context.Context, bucket, originalPr
InclusiveStartFrom: false,
}
- resp, err := client.ListEntries(ctx, request)
+ stream, err := client.ListEntries(ctx, request)
if err != nil {
return fmt.Errorf("list buckets: %v", err)
}
@@ -117,7 +118,18 @@ func (s3a *S3ApiServer) listFilerEntries(ctx context.Context, bucket, originalPr
var counter int
var lastEntryName string
var isTruncated bool
- for _, entry := range resp.Entries {
+
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ break
+ } else {
+ return recvErr
+ }
+ }
+
+ entry := resp.Entry
counter++
if counter > maxKeys {
isTruncated = true
@@ -125,9 +137,11 @@ func (s3a *S3ApiServer) listFilerEntries(ctx context.Context, bucket, originalPr
}
lastEntryName = entry.Name
if entry.IsDirectory {
- commonPrefixes = append(commonPrefixes, PrefixEntry{
- Prefix: fmt.Sprintf("%s%s/", dir, entry.Name),
- })
+ if entry.Name != ".uploads" {
+ commonPrefixes = append(commonPrefixes, PrefixEntry{
+ Prefix: fmt.Sprintf("%s%s/", dir, entry.Name),
+ })
+ }
} else {
contents = append(contents, ListEntry{
Key: fmt.Sprintf("%s%s", dir, entry.Name),
@@ -141,6 +155,7 @@ func (s3a *S3ApiServer) listFilerEntries(ctx context.Context, bucket, originalPr
StorageClass: "STANDARD",
})
}
+
}
response = ListBucketResult{
diff --git a/weed/s3api/s3api_server.go b/weed/s3api/s3api_server.go
index 24458592d..edf634444 100644
--- a/weed/s3api/s3api_server.go
+++ b/weed/s3api/s3api_server.go
@@ -1,12 +1,6 @@
package s3api
import (
- _ "github.com/chrislusf/seaweedfs/weed/filer2/cassandra"
- _ "github.com/chrislusf/seaweedfs/weed/filer2/leveldb"
- _ "github.com/chrislusf/seaweedfs/weed/filer2/memdb"
- _ "github.com/chrislusf/seaweedfs/weed/filer2/mysql"
- _ "github.com/chrislusf/seaweedfs/weed/filer2/postgres"
- _ "github.com/chrislusf/seaweedfs/weed/filer2/redis"
"github.com/gorilla/mux"
"google.golang.org/grpc"
"net/http"
diff --git a/weed/s3api/s3api_xsd_generated.go b/weed/s3api/s3api_xsd_generated.go
index 573c09ede..9d62afc4e 100644
--- a/weed/s3api/s3api_xsd_generated.go
+++ b/weed/s3api/s3api_xsd_generated.go
@@ -675,7 +675,7 @@ type PostResponse struct {
}
type PrefixEntry struct {
- Prefix string `xml:"http://s3.amazonaws.com/doc/2006-03-01/ Prefix"`
+ Prefix string `xml:"Prefix"`
}
type PutObject struct {
diff --git a/weed/sequence/etcd_sequencer.go b/weed/sequence/etcd_sequencer.go
new file mode 100644
index 000000000..1fc378640
--- /dev/null
+++ b/weed/sequence/etcd_sequencer.go
@@ -0,0 +1,296 @@
+package sequence
+
+/*
+Note :
+(1) store the sequence in the ETCD cluster, and local file(sequence.dat)
+(2) batch get the sequences from ETCD cluster, and store the max sequence id in the local file
+(3) the sequence range is : [currentSeqId, maxSeqId), when the currentSeqId >= maxSeqId, fetch the new maxSeqId.
+*/
+
+import (
+ "context"
+ "fmt"
+ "sync"
+ "time"
+
+ "io"
+ "os"
+ "strconv"
+ "strings"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "go.etcd.io/etcd/client"
+)
+
+const (
+ // EtcdKeyPrefix = "/seaweedfs"
+ EtcdKeySequence = "/master/sequence"
+ EtcdContextTimeoutSecond = 100 * time.Second
+ DefaultEtcdSteps uint64 = 500 // internal counter
+ SequencerFileName = "sequencer.dat"
+ FileMaxSequenceLength = 128
+)
+
+type EtcdSequencer struct {
+ sequenceLock sync.Mutex
+
+ // available sequence range : [currentSeqId, maxSeqId)
+ currentSeqId uint64
+ maxSeqId uint64
+
+ keysAPI client.KeysAPI
+ seqFile *os.File
+}
+
+func NewEtcdSequencer(etcdUrls string, metaFolder string) (*EtcdSequencer, error) {
+ file, err := openSequenceFile(metaFolder + "/" + SequencerFileName)
+ if nil != err {
+ return nil, fmt.Errorf("open sequence file fialed, %v", err)
+ }
+
+ cli, err := client.New(client.Config{
+ Endpoints: strings.Split(etcdUrls, ","),
+ Username: "",
+ Password: "",
+ })
+ if err != nil {
+ return nil, err
+ }
+ keysApi := client.NewKeysAPI(cli)
+
+ // TODO: the current sequence id in local file is not used
+ maxValue, _, err := readSequenceFile(file)
+ if err != nil {
+ return nil, fmt.Errorf("read sequence from file failed, %v", err)
+ }
+ glog.V(4).Infof("read sequence from file : %d", maxValue)
+
+ newSeq, err := setMaxSequenceToEtcd(keysApi, maxValue)
+ if err != nil {
+ return nil, err
+ }
+
+ sequencer := &EtcdSequencer{maxSeqId: newSeq,
+ currentSeqId: newSeq,
+ keysAPI: keysApi,
+ seqFile: file,
+ }
+ return sequencer, nil
+}
+
+func (es *EtcdSequencer) NextFileId(count uint64) uint64 {
+ es.sequenceLock.Lock()
+ defer es.sequenceLock.Unlock()
+
+ if (es.currentSeqId + count) >= es.maxSeqId {
+ reqSteps := DefaultEtcdSteps
+ if count > DefaultEtcdSteps {
+ reqSteps += count
+ }
+ maxId, err := batchGetSequenceFromEtcd(es.keysAPI, reqSteps)
+ glog.V(4).Infof("get max sequence id from etcd, %d", maxId)
+ if err != nil {
+ glog.Error(err)
+ return 0
+ }
+ es.currentSeqId, es.maxSeqId = maxId-reqSteps, maxId
+ glog.V(4).Infof("current id : %d, max id : %d", es.currentSeqId, es.maxSeqId)
+
+ if err := writeSequenceFile(es.seqFile, es.maxSeqId, es.currentSeqId); err != nil {
+ glog.Errorf("flush sequence to file failed, %v", err)
+ }
+ }
+
+ ret := es.currentSeqId
+ es.currentSeqId += count
+ return ret
+}
+
+/**
+instead of collecting the max value from volume server,
+the max value should be saved in local config file and ETCD cluster
+*/
+func (es *EtcdSequencer) SetMax(seenValue uint64) {
+ es.sequenceLock.Lock()
+ defer es.sequenceLock.Unlock()
+ if seenValue > es.maxSeqId {
+ maxId, err := setMaxSequenceToEtcd(es.keysAPI, seenValue)
+ if err != nil {
+ glog.Errorf("set Etcd Max sequence failed : %v", err)
+ return
+ }
+ es.currentSeqId, es.maxSeqId = maxId, maxId
+
+ if err := writeSequenceFile(es.seqFile, maxId, maxId); err != nil {
+ glog.Errorf("flush sequence to file failed, %v", err)
+ }
+ }
+}
+
+func (es *EtcdSequencer) GetMax() uint64 {
+ return es.maxSeqId
+}
+
+func (es *EtcdSequencer) Peek() uint64 {
+ return es.currentSeqId
+}
+
+func batchGetSequenceFromEtcd(kvApi client.KeysAPI, step uint64) (uint64, error) {
+ if step <= 0 {
+ return 0, fmt.Errorf("the step must be large than 1")
+ }
+
+ ctx, cancel := context.WithTimeout(context.Background(), EtcdContextTimeoutSecond)
+ var endSeqValue uint64 = 0
+ defer cancel()
+ for {
+ getResp, err := kvApi.Get(ctx, EtcdKeySequence, &client.GetOptions{Recursive: false, Quorum: true})
+ if err != nil {
+ return 0, err
+ }
+ if getResp.Node == nil {
+ continue
+ }
+
+ prevValue := getResp.Node.Value
+ prevSeqValue, err := strconv.ParseUint(prevValue, 10, 64)
+ if err != nil {
+ return 0, fmt.Errorf("get sequence from etcd failed, %v", err)
+ }
+ endSeqValue = prevSeqValue + step
+ endSeqStr := strconv.FormatUint(endSeqValue, 10)
+
+ _, err = kvApi.Set(ctx, EtcdKeySequence, endSeqStr, &client.SetOptions{PrevValue: prevValue})
+ if err == nil {
+ break
+ }
+ glog.Error(err)
+ }
+
+ return endSeqValue, nil
+}
+
+/**
+update the value of the key EtcdKeySequence in ETCD cluster with the parameter of maxSeq,
+when the value of the key EtcdKeySequence is equal to or large than the parameter maxSeq,
+return the value of EtcdKeySequence in the ETCD cluster;
+when the value of the EtcdKeySequence is less than the parameter maxSeq,
+return the value of the parameter maxSeq
+*/
+func setMaxSequenceToEtcd(kvApi client.KeysAPI, maxSeq uint64) (uint64, error) {
+ maxSeqStr := strconv.FormatUint(maxSeq, 10)
+ ctx, cancel := context.WithTimeout(context.Background(), EtcdContextTimeoutSecond)
+ defer cancel()
+
+ for {
+ getResp, err := kvApi.Get(ctx, EtcdKeySequence, &client.GetOptions{Recursive: false, Quorum: true})
+ if err != nil {
+ if ce, ok := err.(client.Error); ok && (ce.Code == client.ErrorCodeKeyNotFound) {
+ _, err := kvApi.Create(ctx, EtcdKeySequence, maxSeqStr)
+ if err == nil {
+ continue
+ }
+ if ce, ok = err.(client.Error); ok && (ce.Code == client.ErrorCodeNodeExist) {
+ continue
+ }
+ return 0, err
+ } else {
+ return 0, err
+ }
+ }
+
+ if getResp.Node == nil {
+ continue
+ }
+ prevSeqStr := getResp.Node.Value
+ prevSeq, err := strconv.ParseUint(prevSeqStr, 10, 64)
+ if err != nil {
+ return 0, err
+ }
+ if prevSeq >= maxSeq {
+ return prevSeq, nil
+ }
+
+ _, err = kvApi.Set(ctx, EtcdKeySequence, maxSeqStr, &client.SetOptions{PrevValue: prevSeqStr})
+ if err != nil {
+ return 0, err
+ }
+ }
+}
+
+func openSequenceFile(file string) (*os.File, error) {
+ _, err := os.Stat(file)
+ if os.IsNotExist(err) {
+ fid, err := os.OpenFile(file, os.O_RDWR|os.O_CREATE, 0644)
+ if err != nil {
+ return nil, err
+ }
+ if err := writeSequenceFile(fid, 1, 0); err != nil {
+ return nil, err
+ }
+ return fid, nil
+ } else {
+ return os.OpenFile(file, os.O_RDWR|os.O_CREATE, 0644)
+ }
+}
+
+/*
+read sequence and step from sequence file
+*/
+func readSequenceFile(file *os.File) (uint64, uint64, error) {
+ sequence := make([]byte, FileMaxSequenceLength)
+ size, err := file.ReadAt(sequence, 0)
+ if (err != nil) && (err != io.EOF) {
+ err := fmt.Errorf("cannot read file %s, %v", file.Name(), err)
+ return 0, 0, err
+ }
+ sequence = sequence[0:size]
+ seqs := strings.Split(string(sequence), ":")
+ maxId, err := strconv.ParseUint(seqs[0], 10, 64)
+ if err != nil {
+ return 0, 0, fmt.Errorf("parse sequence from file failed, %v", err)
+ }
+
+ if len(seqs) > 1 {
+ step, err := strconv.ParseUint(seqs[1], 10, 64)
+ if err != nil {
+ return 0, 0, fmt.Errorf("parse sequence from file failed, %v", err)
+ }
+ return maxId, step, nil
+ }
+
+ return maxId, 0, nil
+}
+
+/**
+write the sequence and step to sequence file
+*/
+func writeSequenceFile(file *os.File, sequence, step uint64) error {
+ _ = step
+ seqStr := fmt.Sprintf("%d:%d", sequence, sequence)
+ if _, err := file.Seek(0, 0); err != nil {
+ err = fmt.Errorf("cannot seek to the beginning of %s: %v", file.Name(), err)
+ return err
+ }
+ if err := file.Truncate(0); err != nil {
+ return fmt.Errorf("truncate sequence file faield : %v", err)
+ }
+ if _, err := file.WriteString(seqStr); err != nil {
+ return fmt.Errorf("write file %s failed, %v", file.Name(), err)
+ }
+ if err := file.Sync(); err != nil {
+ return fmt.Errorf("flush file %s failed, %v", file.Name(), err)
+ }
+ return nil
+}
+
+// the UT helper method
+// func deleteEtcdKey(kvApi client.KeysAPI, key string) error {
+// ctx, cancel := context.WithTimeout(context.Background(), EtcdContextTimeoutSecond)
+// defer cancel()
+// _, err := kvApi.Delete(ctx, key, &client.DeleteOptions{Dir: false})
+// if err != nil {
+// return err
+// }
+// return nil
+// }
diff --git a/weed/sequence/memory_sequencer.go b/weed/sequence/memory_sequencer.go
index d727dc723..e20c29cc7 100644
--- a/weed/sequence/memory_sequencer.go
+++ b/weed/sequence/memory_sequencer.go
@@ -15,12 +15,12 @@ func NewMemorySequencer() (m *MemorySequencer) {
return
}
-func (m *MemorySequencer) NextFileId(count uint64) (uint64, uint64) {
+func (m *MemorySequencer) NextFileId(count uint64) uint64 {
m.sequenceLock.Lock()
defer m.sequenceLock.Unlock()
ret := m.counter
- m.counter += uint64(count)
- return ret, count
+ m.counter += count
+ return ret
}
func (m *MemorySequencer) SetMax(seenValue uint64) {
diff --git a/weed/sequence/sequence.go b/weed/sequence/sequence.go
index fbdc3b8ef..2258d001b 100644
--- a/weed/sequence/sequence.go
+++ b/weed/sequence/sequence.go
@@ -1,7 +1,7 @@
package sequence
type Sequencer interface {
- NextFileId(count uint64) (uint64, uint64)
+ NextFileId(count uint64) uint64
SetMax(uint64)
Peek() uint64
}
diff --git a/weed/server/common.go b/weed/server/common.go
index e02ab38a6..888ddec49 100644
--- a/weed/server/common.go
+++ b/weed/server/common.go
@@ -11,17 +11,18 @@ import (
"strings"
"time"
- "github.com/chrislusf/seaweedfs/weed/storage/needle"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/util"
- _ "github.com/chrislusf/seaweedfs/weed/statik"
"github.com/gorilla/mux"
statik "github.com/rakyll/statik/fs"
+
+ _ "github.com/chrislusf/seaweedfs/weed/statik"
)
var serverStats *stats.ServerStats
@@ -48,10 +49,16 @@ func writeJson(w http.ResponseWriter, r *http.Request, httpStatus int, obj inter
if callback == "" {
w.Header().Set("Content-Type", "application/json")
w.WriteHeader(httpStatus)
+ if httpStatus == http.StatusNotModified {
+ return
+ }
_, err = w.Write(bytes)
} else {
w.Header().Set("Content-Type", "application/javascript")
w.WriteHeader(httpStatus)
+ if httpStatus == http.StatusNotModified {
+ return
+ }
if _, err = w.Write([]uint8(callback)); err != nil {
return
}
@@ -109,6 +116,7 @@ func submitForClientHandler(w http.ResponseWriter, r *http.Request, masterUrl st
}
ar := &operation.VolumeAssignRequest{
Count: count,
+ DataCenter: r.FormValue("dataCenter"),
Replication: r.FormValue("replication"),
Collection: r.FormValue("collection"),
Ttl: r.FormValue("ttl"),
diff --git a/weed/server/filer_grpc_server.go b/weed/server/filer_grpc_server.go
index 8eea2441e..a84feec2d 100644
--- a/weed/server/filer_grpc_server.go
+++ b/weed/server/filer_grpc_server.go
@@ -20,7 +20,7 @@ func (fs *FilerServer) LookupDirectoryEntry(ctx context.Context, req *filer_pb.L
entry, err := fs.filer.FindEntry(ctx, filer2.FullPath(filepath.ToSlash(filepath.Join(req.Directory, req.Name))))
if err != nil {
- return nil, fmt.Errorf("%s not found under %s: %v", req.Name, req.Directory, err)
+ return nil, err
}
return &filer_pb.LookupDirectoryEntryResponse{
@@ -29,27 +29,32 @@ func (fs *FilerServer) LookupDirectoryEntry(ctx context.Context, req *filer_pb.L
IsDirectory: entry.IsDirectory(),
Attributes: filer2.EntryAttributeToPb(entry),
Chunks: entry.Chunks,
+ Extended: entry.Extended,
},
}, nil
}
-func (fs *FilerServer) ListEntries(ctx context.Context, req *filer_pb.ListEntriesRequest) (*filer_pb.ListEntriesResponse, error) {
+func (fs *FilerServer) ListEntries(req *filer_pb.ListEntriesRequest, stream filer_pb.SeaweedFiler_ListEntriesServer) error {
limit := int(req.Limit)
if limit == 0 {
limit = fs.option.DirListingLimit
}
- resp := &filer_pb.ListEntriesResponse{}
+ paginationLimit := filer2.PaginationSize
+ if limit < paginationLimit {
+ paginationLimit = limit
+ }
+
lastFileName := req.StartFromFileName
includeLastFile := req.InclusiveStartFrom
for limit > 0 {
- entries, err := fs.filer.ListDirectoryEntries(ctx, filer2.FullPath(req.Directory), lastFileName, includeLastFile, 1024)
+ entries, err := fs.filer.ListDirectoryEntries(stream.Context(), filer2.FullPath(req.Directory), lastFileName, includeLastFile, paginationLimit)
if err != nil {
- return nil, err
+ return err
}
if len(entries) == 0 {
- return resp, nil
+ return nil
}
includeLastFile = false
@@ -64,22 +69,30 @@ func (fs *FilerServer) ListEntries(ctx context.Context, req *filer_pb.ListEntrie
}
}
- resp.Entries = append(resp.Entries, &filer_pb.Entry{
- Name: entry.Name(),
- IsDirectory: entry.IsDirectory(),
- Chunks: entry.Chunks,
- Attributes: filer2.EntryAttributeToPb(entry),
- })
+ if err := stream.Send(&filer_pb.ListEntriesResponse{
+ Entry: &filer_pb.Entry{
+ Name: entry.Name(),
+ IsDirectory: entry.IsDirectory(),
+ Chunks: entry.Chunks,
+ Attributes: filer2.EntryAttributeToPb(entry),
+ Extended: entry.Extended,
+ },
+ }); err != nil {
+ return err
+ }
limit--
+ if limit == 0 {
+ return nil
+ }
}
- if len(resp.Entries) < 1024 {
+ if len(entries) < paginationLimit {
break
}
}
- return resp, nil
+ return nil
}
func (fs *FilerServer) LookupVolume(ctx context.Context, req *filer_pb.LookupVolumeRequest) (*filer_pb.LookupVolumeResponse, error) {
@@ -95,7 +108,11 @@ func (fs *FilerServer) LookupVolume(ctx context.Context, req *filer_pb.LookupVol
return nil, err
}
var locs []*filer_pb.Location
- for _, loc := range fs.filer.MasterClient.GetLocations(uint32(vid)) {
+ locations, found := fs.filer.MasterClient.GetLocations(uint32(vid))
+ if !found {
+ continue
+ }
+ for _, loc := range locations {
locs = append(locs, &filer_pb.Location{
Url: loc.Url,
PublicUrl: loc.PublicUrl,
@@ -125,7 +142,7 @@ func (fs *FilerServer) CreateEntry(ctx context.Context, req *filer_pb.CreateEntr
})
if err == nil {
- fs.filer.DeleteChunks(fullpath, garbages)
+ fs.filer.DeleteChunks(garbages)
}
return &filer_pb.CreateEntryResponse{}, err
@@ -147,12 +164,14 @@ func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntr
newEntry := &filer2.Entry{
FullPath: filer2.FullPath(filepath.ToSlash(filepath.Join(req.Directory, req.Entry.Name))),
Attr: entry.Attr,
+ Extended: req.Entry.Extended,
Chunks: chunks,
}
- glog.V(3).Infof("updating %s: %+v, chunks %d: %v => %+v, chunks %d: %v",
+ glog.V(3).Infof("updating %s: %+v, chunks %d: %v => %+v, chunks %d: %v, extended: %v => %v",
fullpath, entry.Attr, len(entry.Chunks), entry.Chunks,
- req.Entry.Attributes, len(req.Entry.Chunks), req.Entry.Chunks)
+ req.Entry.Attributes, len(req.Entry.Chunks), req.Entry.Chunks,
+ entry.Extended, req.Entry.Extended)
if req.Entry.Attributes != nil {
if req.Entry.Attributes.Mtime != 0 {
@@ -174,8 +193,8 @@ func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntr
}
if err = fs.filer.UpdateEntry(ctx, entry, newEntry); err == nil {
- fs.filer.DeleteChunks(entry.FullPath, unusedChunks)
- fs.filer.DeleteChunks(entry.FullPath, garbages)
+ fs.filer.DeleteChunks(unusedChunks)
+ fs.filer.DeleteChunks(garbages)
}
fs.filer.NotifyUpdateEvent(entry, newEntry, true)
@@ -184,7 +203,7 @@ func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntr
}
func (fs *FilerServer) DeleteEntry(ctx context.Context, req *filer_pb.DeleteEntryRequest) (resp *filer_pb.DeleteEntryResponse, err error) {
- err = fs.filer.DeleteEntryMetaAndData(ctx, filer2.FullPath(filepath.ToSlash(filepath.Join(req.Directory, req.Name))), req.IsRecursive, req.IsDeleteData)
+ err = fs.filer.DeleteEntryMetaAndData(ctx, filer2.FullPath(filepath.ToSlash(filepath.Join(req.Directory, req.Name))), req.IsRecursive, req.IgnoreRecursiveError, req.IsDeleteData)
return &filer_pb.DeleteEntryResponse{}, err
}
diff --git a/weed/server/filer_grpc_server_rename.go b/weed/server/filer_grpc_server_rename.go
index 7142f7606..dfa59e7fe 100644
--- a/weed/server/filer_grpc_server_rename.go
+++ b/weed/server/filer_grpc_server_rename.go
@@ -73,11 +73,11 @@ func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, oldParent filer
return err
}
- println("found", len(entries), "entries under", currentDirPath)
+ // println("found", len(entries), "entries under", currentDirPath)
for _, item := range entries {
lastFileName = item.Name()
- println("processing", lastFileName)
+ // println("processing", lastFileName)
err := fs.moveEntry(ctx, currentDirPath, item, newDirPath, item.Name(), events)
if err != nil {
return err
@@ -113,7 +113,7 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, oldParent filer2.FullP
}
// delete old entry
- deleteErr := fs.filer.DeleteEntryMetaAndData(ctx, oldPath, false, false)
+ deleteErr := fs.filer.DeleteEntryMetaAndData(ctx, oldPath, false, false, false)
if deleteErr != nil {
return deleteErr
}
diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go
index b9e6aa23d..41ba81366 100644
--- a/weed/server/filer_server.go
+++ b/weed/server/filer_server.go
@@ -15,12 +15,13 @@ import (
"github.com/chrislusf/seaweedfs/weed/filer2"
_ "github.com/chrislusf/seaweedfs/weed/filer2/cassandra"
+ _ "github.com/chrislusf/seaweedfs/weed/filer2/etcd"
_ "github.com/chrislusf/seaweedfs/weed/filer2/leveldb"
_ "github.com/chrislusf/seaweedfs/weed/filer2/leveldb2"
- _ "github.com/chrislusf/seaweedfs/weed/filer2/memdb"
_ "github.com/chrislusf/seaweedfs/weed/filer2/mysql"
_ "github.com/chrislusf/seaweedfs/weed/filer2/postgres"
_ "github.com/chrislusf/seaweedfs/weed/filer2/redis"
+ _ "github.com/chrislusf/seaweedfs/weed/filer2/tikv"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/notification"
_ "github.com/chrislusf/seaweedfs/weed/notification/aws_sqs"
diff --git a/weed/server/filer_server_handlers_read.go b/weed/server/filer_server_handlers_read.go
index 0edf501a8..ba21298ba 100644
--- a/weed/server/filer_server_handlers_read.go
+++ b/weed/server/filer_server_handlers_read.go
@@ -32,10 +32,15 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request,
fs.listDirectoryHandler(w, r)
return
}
- glog.V(1).Infof("Not found %s: %v", path, err)
-
- stats.FilerRequestCounter.WithLabelValues("read.notfound").Inc()
- w.WriteHeader(http.StatusNotFound)
+ if err == filer2.ErrNotFound {
+ glog.V(1).Infof("Not found %s: %v", path, err)
+ stats.FilerRequestCounter.WithLabelValues("read.notfound").Inc()
+ w.WriteHeader(http.StatusNotFound)
+ } else {
+ glog.V(0).Infof("Internal %s: %v", path, err)
+ stats.FilerRequestCounter.WithLabelValues("read.internalerror").Inc()
+ w.WriteHeader(http.StatusInternalServerError)
+ }
return
}
diff --git a/weed/server/filer_server_handlers_write.go b/weed/server/filer_server_handlers_write.go
index 0bf1218ce..236e7027d 100644
--- a/weed/server/filer_server_handlers_write.go
+++ b/weed/server/filer_server_handlers_write.go
@@ -149,6 +149,16 @@ func (fs *FilerServer) updateFilerStore(ctx context.Context, r *http.Request, w
stats.FilerRequestHistogram.WithLabelValues("postStoreWrite").Observe(time.Since(start).Seconds())
}()
+ modeStr := r.URL.Query().Get("mode")
+ if modeStr == "" {
+ modeStr = "0660"
+ }
+ mode, err := strconv.ParseUint(modeStr, 8, 32)
+ if err != nil {
+ glog.Errorf("Invalid mode format: %s, use 0660 by default", modeStr)
+ mode = 0660
+ }
+
path := r.URL.Path
if strings.HasSuffix(path, "/") {
if ret.Name != "" {
@@ -165,7 +175,7 @@ func (fs *FilerServer) updateFilerStore(ctx context.Context, r *http.Request, w
Attr: filer2.Attr{
Mtime: time.Now(),
Crtime: crTime,
- Mode: 0660,
+ Mode: os.FileMode(mode),
Uid: OS_UID,
Gid: OS_GID,
Replication: replication,
@@ -184,7 +194,7 @@ func (fs *FilerServer) updateFilerStore(ctx context.Context, r *http.Request, w
}
// glog.V(4).Infof("saving %s => %+v", path, entry)
if dbErr := fs.filer.CreateEntry(ctx, entry); dbErr != nil {
- fs.filer.DeleteChunks(entry.FullPath, entry.Chunks)
+ fs.filer.DeleteChunks(entry.Chunks)
glog.V(0).Infof("failing to write %s to filer server : %v", path, dbErr)
writeJsonError(w, r, http.StatusInternalServerError, dbErr)
err = dbErr
@@ -269,14 +279,22 @@ func (fs *FilerServer) uploadToVolumeServer(r *http.Request, u *url.URL, auth se
// curl -X DELETE http://localhost:8888/path/to
// curl -X DELETE http://localhost:8888/path/to?recursive=true
+// curl -X DELETE http://localhost:8888/path/to?recursive=true&ignoreRecursiveError=true
+// curl -X DELETE http://localhost:8888/path/to?recursive=true&skipChunkDeletion=true
func (fs *FilerServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
isRecursive := r.FormValue("recursive") == "true"
+ ignoreRecursiveError := r.FormValue("ignoreRecursiveError") == "true"
+ skipChunkDeletion := r.FormValue("skipChunkDeletion") == "true"
- err := fs.filer.DeleteEntryMetaAndData(context.Background(), filer2.FullPath(r.URL.Path), isRecursive, true)
+ err := fs.filer.DeleteEntryMetaAndData(context.Background(), filer2.FullPath(r.URL.Path), isRecursive, ignoreRecursiveError, !skipChunkDeletion)
if err != nil {
glog.V(1).Infoln("deleting", r.URL.Path, ":", err.Error())
- writeJsonError(w, r, http.StatusInternalServerError, err)
+ httpStatus := http.StatusInternalServerError
+ if err == filer2.ErrNotFound {
+ httpStatus = http.StatusNotFound
+ }
+ writeJsonError(w, r, httpStatus, err)
return
}
diff --git a/weed/server/filer_server_handlers_write_autochunk.go b/weed/server/filer_server_handlers_write_autochunk.go
index 492b55943..8ff7ab2c0 100644
--- a/weed/server/filer_server_handlers_write_autochunk.go
+++ b/weed/server/filer_server_handlers_write_autochunk.go
@@ -123,7 +123,7 @@ func (fs *FilerServer) doAutoChunk(ctx context.Context, w http.ResponseWriter, r
// upload the chunk to the volume server
chunkName := fileName + "_chunk_" + strconv.FormatInt(int64(len(fileChunks)+1), 10)
- uploadErr := fs.doUpload(urlLocation, w, r, chunkBuf[0:chunkBufOffset], chunkName, "application/octet-stream", fileId, auth)
+ uploadErr := fs.doUpload(urlLocation, w, r, chunkBuf[0:chunkBufOffset], chunkName, "", fileId, auth)
if uploadErr != nil {
return nil, uploadErr
}
@@ -177,7 +177,7 @@ func (fs *FilerServer) doAutoChunk(ctx context.Context, w http.ResponseWriter, r
Chunks: fileChunks,
}
if dbErr := fs.filer.CreateEntry(ctx, entry); dbErr != nil {
- fs.filer.DeleteChunks(entry.FullPath, entry.Chunks)
+ fs.filer.DeleteChunks(entry.Chunks)
replyerr = dbErr
filerResult.Error = dbErr.Error()
glog.V(0).Infof("failing to write %s to filer server : %v", path, dbErr)
diff --git a/weed/server/filer_ui/breadcrumb.go b/weed/server/filer_ui/breadcrumb.go
index 55a1909a8..2f0df7f91 100644
--- a/weed/server/filer_ui/breadcrumb.go
+++ b/weed/server/filer_ui/breadcrumb.go
@@ -14,10 +14,14 @@ func ToBreadcrumb(fullpath string) (crumbs []Breadcrumb) {
parts := strings.Split(fullpath, "/")
for i := 0; i < len(parts); i++ {
- crumbs = append(crumbs, Breadcrumb{
- Name: parts[i] + "/",
+ crumb := Breadcrumb{
+ Name: parts[i] + " /",
Link: "/" + filepath.ToSlash(filepath.Join(parts[0:i+1]...)),
- })
+ }
+ if !strings.HasSuffix(crumb.Link, "/") {
+ crumb.Link += "/"
+ }
+ crumbs = append(crumbs, crumb)
}
return
diff --git a/weed/server/filer_ui/templates.go b/weed/server/filer_ui/templates.go
index 884798936..e532b27e2 100644
--- a/weed/server/filer_ui/templates.go
+++ b/weed/server/filer_ui/templates.go
@@ -50,7 +50,7 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`
{{ range $entry := .Breadcrumbs }}
-
+
{{ $entry.Name }}
{{ end }}
@@ -78,20 +78,19 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`
{{end}}
-
+
{{if $entry.IsDirectory}}
{{else}}
- {{ $entry.Mime }}
+ {{ $entry.Mime }}
{{end}}
-
+
{{if $entry.IsDirectory}}
{{else}}
- {{ $entry.Size | humanizeBytes }}
-
+ {{ $entry.Size | humanizeBytes }}
{{end}}
-
+
{{ $entry.Timestamp.Format "2006-01-02 15:04" }}
diff --git a/weed/server/master_grpc_server.go b/weed/server/master_grpc_server.go
index e0d1fd174..fcfd98f7b 100644
--- a/weed/server/master_grpc_server.go
+++ b/weed/server/master_grpc_server.go
@@ -9,6 +9,7 @@ import (
"github.com/chrislusf/raft"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/topology"
"google.golang.org/grpc/peer"
@@ -49,6 +50,11 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
for {
heartbeat, err := stream.Recv()
if err != nil {
+ if dn != nil {
+ glog.Warningf("SendHeartbeat.Recv server %s:%d : %v", dn.Ip, dn.Port, err)
+ } else {
+ glog.Warningf("SendHeartbeat.Recv: %v", err)
+ }
return err
}
@@ -71,8 +77,12 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
int64(heartbeat.MaxVolumeCount))
glog.V(0).Infof("added volume server %v:%d", heartbeat.GetIp(), heartbeat.GetPort())
if err := stream.Send(&master_pb.HeartbeatResponse{
- VolumeSizeLimit: uint64(ms.option.VolumeSizeLimitMB) * 1024 * 1024,
+ VolumeSizeLimit: uint64(ms.option.VolumeSizeLimitMB) * 1024 * 1024,
+ MetricsAddress: ms.option.MetricsAddress,
+ MetricsIntervalSeconds: uint32(ms.option.MetricsIntervalSec),
+ StorageBackends: backend.ToPbStorageBackends(),
}); err != nil {
+ glog.Warningf("SendHeartbeat.Send volume size to %s:%d %v", dn.Ip, dn.Port, err)
return err
}
}
@@ -154,13 +164,13 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
// tell the volume servers about the leader
newLeader, err := t.Leader()
if err != nil {
+ glog.Warningf("SendHeartbeat find leader: %v", err)
return err
}
if err := stream.Send(&master_pb.HeartbeatResponse{
- Leader: newLeader,
- MetricsAddress: ms.option.MetricsAddress,
- MetricsIntervalSeconds: uint32(ms.option.MetricsIntervalSec),
+ Leader: newLeader,
}); err != nil {
+ glog.Warningf("SendHeartbeat.Send response to to %s:%d %v", dn.Ip, dn.Port, err)
return err
}
}
@@ -176,7 +186,7 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ
}
if !ms.Topo.IsLeader() {
- return raft.NotLeaderError
+ return ms.informNewLeader(stream)
}
// remember client address
@@ -236,7 +246,7 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ
}
case <-ticker.C:
if !ms.Topo.IsLeader() {
- return raft.NotLeaderError
+ return ms.informNewLeader(stream)
}
case <-stopChan:
return nil
@@ -245,3 +255,17 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ
return nil
}
+
+func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedServer) error {
+ leader, err := ms.Topo.Leader()
+ if err != nil {
+ glog.Errorf("topo leader: %v", err)
+ return raft.NotLeaderError
+ }
+ if err := stream.Send(&master_pb.VolumeLocation{
+ Leader: leader,
+ }); err != nil {
+ return err
+ }
+ return nil
+}
diff --git a/weed/server/master_grpc_server_collection.go b/weed/server/master_grpc_server_collection.go
index a50cfa192..f8e0785f6 100644
--- a/weed/server/master_grpc_server_collection.go
+++ b/weed/server/master_grpc_server_collection.go
@@ -57,7 +57,7 @@ func (ms *MasterServer) doDeleteNormalCollection(collectionName string) error {
}
for _, server := range collection.ListVolumeServers() {
- err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOpiton, func(client volume_server_pb.VolumeServerClient) error {
+ err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(context.Background(), &volume_server_pb.DeleteCollectionRequest{
Collection: collectionName,
})
@@ -77,7 +77,7 @@ func (ms *MasterServer) doDeleteEcCollection(collectionName string) error {
listOfEcServers := ms.Topo.ListEcServersByCollection(collectionName)
for _, server := range listOfEcServers {
- err := operation.WithVolumeServerClient(server, ms.grpcDialOpiton, func(client volume_server_pb.VolumeServerClient) error {
+ err := operation.WithVolumeServerClient(server, ms.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(context.Background(), &volume_server_pb.DeleteCollectionRequest{
Collection: collectionName,
})
diff --git a/weed/server/master_grpc_server_volume.go b/weed/server/master_grpc_server_volume.go
index 19064bcde..856c07890 100644
--- a/weed/server/master_grpc_server_volume.go
+++ b/weed/server/master_grpc_server_volume.go
@@ -5,10 +5,11 @@ import (
"fmt"
"github.com/chrislusf/raft"
+
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/security"
- "github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/topology"
)
@@ -52,7 +53,7 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
if req.Replication == "" {
req.Replication = ms.option.DefaultReplicaPlacement
}
- replicaPlacement, err := storage.NewReplicaPlacementFromString(req.Replication)
+ replicaPlacement, err := super_block.NewReplicaPlacementFromString(req.Replication)
if err != nil {
return nil, err
}
@@ -62,13 +63,14 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
}
option := &topology.VolumeGrowOption{
- Collection: req.Collection,
- ReplicaPlacement: replicaPlacement,
- Ttl: ttl,
- Prealloacte: ms.preallocateSize,
- DataCenter: req.DataCenter,
- Rack: req.Rack,
- DataNode: req.DataNode,
+ Collection: req.Collection,
+ ReplicaPlacement: replicaPlacement,
+ Ttl: ttl,
+ Prealloacte: ms.preallocateSize,
+ DataCenter: req.DataCenter,
+ Rack: req.Rack,
+ DataNode: req.DataNode,
+ MemoryMapMaxSizeMb: req.MemoryMapMaxSizeMb,
}
if !ms.Topo.HasWritableVolume(option) {
@@ -77,7 +79,7 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
}
ms.vgLock.Lock()
if !ms.Topo.HasWritableVolume(option) {
- if _, err = ms.vg.AutomaticGrowByType(option, ms.grpcDialOpiton, ms.Topo); err != nil {
+ if _, err = ms.vg.AutomaticGrowByType(option, ms.grpcDialOption, ms.Topo, int(req.WritableVolumeCount)); err != nil {
ms.vgLock.Unlock()
return nil, fmt.Errorf("Cannot grow volume group! %v", err)
}
@@ -107,7 +109,7 @@ func (ms *MasterServer) Statistics(ctx context.Context, req *master_pb.Statistic
if req.Replication == "" {
req.Replication = ms.option.DefaultReplicaPlacement
}
- replicaPlacement, err := storage.NewReplicaPlacementFromString(req.Replication)
+ replicaPlacement, err := super_block.NewReplicaPlacementFromString(req.Replication)
if err != nil {
return nil, err
}
diff --git a/weed/server/master_server.go b/weed/server/master_server.go
index 3689b5495..33a5129da 100644
--- a/weed/server/master_server.go
+++ b/weed/server/master_server.go
@@ -1,9 +1,8 @@
package weed_server
import (
+ "context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/shell"
- "google.golang.org/grpc"
"net/http"
"net/http/httputil"
"net/url"
@@ -19,10 +18,18 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/sequence"
+ "github.com/chrislusf/seaweedfs/weed/shell"
"github.com/chrislusf/seaweedfs/weed/topology"
"github.com/chrislusf/seaweedfs/weed/util"
+ "github.com/chrislusf/seaweedfs/weed/wdclient"
"github.com/gorilla/mux"
"github.com/spf13/viper"
+ "google.golang.org/grpc"
+)
+
+const (
+ SequencerType = "master.sequencer.type"
+ SequencerEtcdUrls = "master.sequencer.sequencer_etcd_urls"
)
type MasterOption struct {
@@ -55,10 +62,12 @@ type MasterServer struct {
clientChansLock sync.RWMutex
clientChans map[string]chan *master_pb.VolumeLocation
- grpcDialOpiton grpc.DialOption
+ grpcDialOption grpc.DialOption
+
+ MasterClient *wdclient.MasterClient
}
-func NewMasterServer(r *mux.Router, option *MasterOption) *MasterServer {
+func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *MasterServer {
v := viper.GetViper()
signingKey := v.GetString("jwt.signing.key")
@@ -73,14 +82,21 @@ func NewMasterServer(r *mux.Router, option *MasterOption) *MasterServer {
if option.VolumePreallocate {
preallocateSize = int64(option.VolumeSizeLimitMB) * (1 << 20)
}
+
+ grpcDialOption := security.LoadClientTLS(v.Sub("grpc"), "master")
ms := &MasterServer{
option: option,
preallocateSize: preallocateSize,
clientChans: make(map[string]chan *master_pb.VolumeLocation),
- grpcDialOpiton: security.LoadClientTLS(v.Sub("grpc"), "master"),
+ grpcDialOption: grpcDialOption,
+ MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "master", peers),
}
ms.bounedLeaderChan = make(chan int, 16)
- seq := sequence.NewMemorySequencer()
+
+ seq := ms.createSequencer(option)
+ if nil == seq {
+ glog.Fatalf("create sequencer failed.")
+ }
ms.Topo = topology.NewTopology("topo", seq, uint64(ms.option.VolumeSizeLimitMB)*1024*1024, ms.option.PulseSeconds)
ms.vg = topology.NewDefaultVolumeGrowth()
glog.V(0).Infoln("Volume Size Limit is", ms.option.VolumeSizeLimitMB, "MB")
@@ -92,7 +108,7 @@ func NewMasterServer(r *mux.Router, option *MasterOption) *MasterServer {
r.HandleFunc("/", ms.proxyToLeader(ms.uiStatusHandler))
r.HandleFunc("/ui/index.html", ms.uiStatusHandler)
r.HandleFunc("/dir/assign", ms.proxyToLeader(ms.guard.WhiteList(ms.dirAssignHandler)))
- r.HandleFunc("/dir/lookup", ms.proxyToLeader(ms.guard.WhiteList(ms.dirLookupHandler)))
+ r.HandleFunc("/dir/lookup", ms.guard.WhiteList(ms.dirLookupHandler))
r.HandleFunc("/dir/status", ms.proxyToLeader(ms.guard.WhiteList(ms.dirStatusHandler)))
r.HandleFunc("/col/delete", ms.proxyToLeader(ms.guard.WhiteList(ms.collectionDeleteHandler)))
r.HandleFunc("/vol/grow", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeGrowHandler)))
@@ -102,10 +118,10 @@ func NewMasterServer(r *mux.Router, option *MasterOption) *MasterServer {
r.HandleFunc("/stats/health", ms.guard.WhiteList(statsHealthHandler))
r.HandleFunc("/stats/counter", ms.guard.WhiteList(statsCounterHandler))
r.HandleFunc("/stats/memory", ms.guard.WhiteList(statsMemoryHandler))
- r.HandleFunc("/{fileId}", ms.proxyToLeader(ms.redirectHandler))
+ r.HandleFunc("/{fileId}", ms.redirectHandler)
}
- ms.Topo.StartRefreshWritableVolumes(ms.grpcDialOpiton, ms.option.GarbageThreshold, ms.preallocateSize)
+ ms.Topo.StartRefreshWritableVolumes(ms.grpcDialOption, ms.option.GarbageThreshold, ms.preallocateSize)
ms.startAdminScripts()
@@ -158,23 +174,28 @@ func (ms *MasterServer) proxyToLeader(f func(w http.ResponseWriter, r *http.Requ
proxy.Transport = util.Transport
proxy.ServeHTTP(w, r)
} else {
- //drop it to the floor
- //writeJsonError(w, r, errors.New(ms.Topo.RaftServer.Name()+" does not know Leader yet:"+ms.Topo.RaftServer.Leader()))
+ // drop it to the floor
+ // writeJsonError(w, r, errors.New(ms.Topo.RaftServer.Name()+" does not know Leader yet:"+ms.Topo.RaftServer.Leader()))
}
}
}
func (ms *MasterServer) startAdminScripts() {
+ var err error
+
v := viper.GetViper()
adminScripts := v.GetString("master.maintenance.scripts")
- v.SetDefault("master.maintenance.sleep_minutes", 17)
- sleepMinutes := v.GetInt("master.maintenance.sleep_minutes")
-
glog.V(0).Infof("adminScripts:\n%v", adminScripts)
if adminScripts == "" {
return
}
+ v.SetDefault("master.maintenance.sleep_minutes", 17)
+ sleepMinutes := v.GetInt("master.maintenance.sleep_minutes")
+
+ v.SetDefault("master.filer.default_filer_url", "http://localhost:8888/")
+ filerURL := v.GetString("master.filer.default_filer_url")
+
scriptLines := strings.Split(adminScripts, "\n")
masterAddress := "localhost:" + strconv.Itoa(ms.option.Port)
@@ -182,9 +203,12 @@ func (ms *MasterServer) startAdminScripts() {
var shellOptions shell.ShellOptions
shellOptions.GrpcDialOption = security.LoadClientTLS(viper.Sub("grpc"), "master")
shellOptions.Masters = &masterAddress
- shellOptions.FilerHost = "localhost"
- shellOptions.FilerPort = 8888
- shellOptions.Directory = "/"
+
+ shellOptions.FilerHost, shellOptions.FilerPort, shellOptions.Directory, err = util.ParseFilerUrl(filerURL)
+ if err != nil {
+ glog.V(0).Infof("failed to parse master.filer.default_filer_urll=%s : %v\n", filerURL, err)
+ return
+ }
commandEnv := shell.NewCommandEnv(shellOptions)
@@ -223,3 +247,24 @@ func (ms *MasterServer) startAdminScripts() {
}
}()
}
+
+func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer {
+ var seq sequence.Sequencer
+ v := viper.GetViper()
+ seqType := strings.ToLower(v.GetString(SequencerType))
+ glog.V(1).Infof("[%s] : [%s]", SequencerType, seqType)
+ switch strings.ToLower(seqType) {
+ case "etcd":
+ var err error
+ urls := v.GetString(SequencerEtcdUrls)
+ glog.V(0).Infof("[%s] : [%s]", SequencerEtcdUrls, urls)
+ seq, err = sequence.NewEtcdSequencer(urls, option.MetaFolder)
+ if err != nil {
+ glog.Error(err)
+ seq = nil
+ }
+ default:
+ seq = sequence.NewMemorySequencer()
+ }
+ return seq
+}
diff --git a/weed/server/master_server_handlers.go b/weed/server/master_server_handlers.go
index 5c7ff41cf..514d86800 100644
--- a/weed/server/master_server_handlers.go
+++ b/weed/server/master_server_handlers.go
@@ -22,21 +22,7 @@ func (ms *MasterServer) lookupVolumeId(vids []string, collection string) (volume
if _, ok := volumeLocations[vid]; ok {
continue
}
- volumeId, err := needle.NewVolumeId(vid)
- if err == nil {
- machines := ms.Topo.Lookup(collection, volumeId)
- if machines != nil {
- var ret []operation.Location
- for _, dn := range machines {
- ret = append(ret, operation.Location{Url: dn.Url(), PublicUrl: dn.PublicUrl})
- }
- volumeLocations[vid] = operation.LookupResult{VolumeId: vid, Locations: ret}
- } else {
- volumeLocations[vid] = operation.LookupResult{VolumeId: vid, Error: fmt.Sprintf("volumeId %s not found.", vid)}
- }
- } else {
- volumeLocations[vid] = operation.LookupResult{VolumeId: vid, Error: fmt.Sprintf("Unknown volumeId format: %s", vid)}
- }
+ volumeLocations[vid] = ms.findVolumeLocation(collection, vid)
}
return
}
@@ -59,12 +45,10 @@ func (ms *MasterServer) dirLookupHandler(w http.ResponseWriter, r *http.Request)
vid = fileId[0:commaSep]
}
}
- vids := []string{vid}
collection := r.FormValue("collection") //optional, but can be faster if too many collections
- volumeLocations := ms.lookupVolumeId(vids, collection)
- location := volumeLocations[vid]
+ location := ms.findVolumeLocation(collection, vid)
httpStatus := http.StatusOK
- if location.Error != "" {
+ if location.Error != "" || location.Locations == nil {
httpStatus = http.StatusNotFound
} else {
forRead := r.FormValue("read")
@@ -74,6 +58,41 @@ func (ms *MasterServer) dirLookupHandler(w http.ResponseWriter, r *http.Request)
writeJsonQuiet(w, r, httpStatus, location)
}
+// findVolumeLocation finds the volume location from master topo if it is leader,
+// or from master client if not leader
+func (ms *MasterServer) findVolumeLocation(collection, vid string) operation.LookupResult {
+ var locations []operation.Location
+ var err error
+ if ms.Topo.IsLeader() {
+ volumeId, newVolumeIdErr := needle.NewVolumeId(vid)
+ if newVolumeIdErr != nil {
+ err = fmt.Errorf("Unknown volume id %s", vid)
+ } else {
+ machines := ms.Topo.Lookup(collection, volumeId)
+ for _, loc := range machines {
+ locations = append(locations, operation.Location{Url: loc.Url(), PublicUrl: loc.PublicUrl})
+ }
+ if locations == nil {
+ err = fmt.Errorf("volume id %s not found", vid)
+ }
+ }
+ } else {
+ machines, getVidLocationsErr := ms.MasterClient.GetVidLocations(vid)
+ for _, loc := range machines {
+ locations = append(locations, operation.Location{Url: loc.Url, PublicUrl: loc.PublicUrl})
+ }
+ err = getVidLocationsErr
+ }
+ ret := operation.LookupResult{
+ VolumeId: vid,
+ Locations: locations,
+ }
+ if err != nil {
+ ret.Error = err.Error()
+ }
+ return ret
+}
+
func (ms *MasterServer) dirAssignHandler(w http.ResponseWriter, r *http.Request) {
stats.AssignRequest()
requestedCount, e := strconv.ParseUint(r.FormValue("count"), 10, 64)
@@ -81,6 +100,11 @@ func (ms *MasterServer) dirAssignHandler(w http.ResponseWriter, r *http.Request)
requestedCount = 1
}
+ writableVolumeCount, e := strconv.Atoi(r.FormValue("writableVolumeCount"))
+ if e != nil {
+ writableVolumeCount = 0
+ }
+
option, err := ms.getVolumeGrowOption(r)
if err != nil {
writeJsonQuiet(w, r, http.StatusNotAcceptable, operation.AssignResult{Error: err.Error()})
@@ -95,7 +119,7 @@ func (ms *MasterServer) dirAssignHandler(w http.ResponseWriter, r *http.Request)
ms.vgLock.Lock()
defer ms.vgLock.Unlock()
if !ms.Topo.HasWritableVolume(option) {
- if _, err = ms.vg.AutomaticGrowByType(option, ms.grpcDialOpiton, ms.Topo); err != nil {
+ if _, err = ms.vg.AutomaticGrowByType(option, ms.grpcDialOption, ms.Topo, writableVolumeCount); err != nil {
writeJsonError(w, r, http.StatusInternalServerError,
fmt.Errorf("Cannot grow volume group! %v", err))
return
diff --git a/weed/server/master_server_handlers_admin.go b/weed/server/master_server_handlers_admin.go
index 343bcb8da..2965a4863 100644
--- a/weed/server/master_server_handlers_admin.go
+++ b/weed/server/master_server_handlers_admin.go
@@ -2,7 +2,6 @@ package weed_server
import (
"context"
- "errors"
"fmt"
"math/rand"
"net/http"
@@ -11,20 +10,22 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
- "github.com/chrislusf/seaweedfs/weed/storage"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend/memory_map"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/topology"
"github.com/chrislusf/seaweedfs/weed/util"
)
func (ms *MasterServer) collectionDeleteHandler(w http.ResponseWriter, r *http.Request) {
- collection, ok := ms.Topo.FindCollection(r.FormValue("collection"))
+ collectionName := r.FormValue("collection")
+ collection, ok := ms.Topo.FindCollection(collectionName)
if !ok {
- writeJsonError(w, r, http.StatusBadRequest, fmt.Errorf("collection %s does not exist", r.FormValue("collection")))
+ writeJsonError(w, r, http.StatusBadRequest, fmt.Errorf("collection %s does not exist", collectionName))
return
}
for _, server := range collection.ListVolumeServers() {
- err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOpiton, func(client volume_server_pb.VolumeServerClient) error {
+ err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(context.Background(), &volume_server_pb.DeleteCollectionRequest{
Collection: collection.Name,
})
@@ -35,7 +36,10 @@ func (ms *MasterServer) collectionDeleteHandler(w http.ResponseWriter, r *http.R
return
}
}
- ms.Topo.DeleteCollection(r.FormValue("collection"))
+ ms.Topo.DeleteCollection(collectionName)
+
+ w.WriteHeader(http.StatusNoContent)
+ return
}
func (ms *MasterServer) dirStatusHandler(w http.ResponseWriter, r *http.Request) {
@@ -53,11 +57,12 @@ func (ms *MasterServer) volumeVacuumHandler(w http.ResponseWriter, r *http.Reque
gcThreshold, err = strconv.ParseFloat(gcString, 32)
if err != nil {
glog.V(0).Infof("garbageThreshold %s is not a valid float number: %v", gcString, err)
+ writeJsonError(w, r, http.StatusNotAcceptable, fmt.Errorf("garbageThreshold %s is not a valid float number", gcString))
return
}
}
glog.Infoln("garbageThreshold =", gcThreshold)
- ms.Topo.Vacuum(ms.grpcDialOpiton, gcThreshold, ms.preallocateSize)
+ ms.Topo.Vacuum(ms.grpcDialOption, gcThreshold, ms.preallocateSize)
ms.dirStatusHandler(w, r)
}
@@ -68,17 +73,17 @@ func (ms *MasterServer) volumeGrowHandler(w http.ResponseWriter, r *http.Request
writeJsonError(w, r, http.StatusNotAcceptable, err)
return
}
- if err == nil {
- if count, err = strconv.Atoi(r.FormValue("count")); err == nil {
- if ms.Topo.FreeSpace() < int64(count*option.ReplicaPlacement.GetCopyCount()) {
- err = fmt.Errorf("only %d volumes left, not enough for %d", ms.Topo.FreeSpace(), count*option.ReplicaPlacement.GetCopyCount())
- } else {
- count, err = ms.vg.GrowByCountAndType(ms.grpcDialOpiton, count, option, ms.Topo)
- }
+
+ if count, err = strconv.Atoi(r.FormValue("count")); err == nil {
+ if ms.Topo.FreeSpace() < int64(count*option.ReplicaPlacement.GetCopyCount()) {
+ err = fmt.Errorf("only %d volumes left, not enough for %d", ms.Topo.FreeSpace(), count*option.ReplicaPlacement.GetCopyCount())
} else {
- err = errors.New("parameter count is not found")
+ count, err = ms.vg.GrowByCountAndType(ms.grpcDialOption, count, option, ms.Topo)
}
+ } else {
+ err = fmt.Errorf("can not parse parameter count %s", r.FormValue("count"))
}
+
if err != nil {
writeJsonError(w, r, http.StatusNotAcceptable, err)
} else {
@@ -95,23 +100,19 @@ func (ms *MasterServer) volumeStatusHandler(w http.ResponseWriter, r *http.Reque
func (ms *MasterServer) redirectHandler(w http.ResponseWriter, r *http.Request) {
vid, _, _, _, _ := parseURLPath(r.URL.Path)
- volumeId, err := needle.NewVolumeId(vid)
- if err != nil {
- debug("parsing error:", err, r.URL.Path)
- return
- }
collection := r.FormValue("collection")
- machines := ms.Topo.Lookup(collection, volumeId)
- if machines != nil && len(machines) > 0 {
+ location := ms.findVolumeLocation(collection, vid)
+ if location.Error == "" {
+ loc := location.Locations[rand.Intn(len(location.Locations))]
var url string
if r.URL.RawQuery != "" {
- url = util.NormalizeUrl(machines[rand.Intn(len(machines))].PublicUrl) + r.URL.Path + "?" + r.URL.RawQuery
+ url = util.NormalizeUrl(loc.PublicUrl) + r.URL.Path + "?" + r.URL.RawQuery
} else {
- url = util.NormalizeUrl(machines[rand.Intn(len(machines))].PublicUrl) + r.URL.Path
+ url = util.NormalizeUrl(loc.PublicUrl) + r.URL.Path
}
http.Redirect(w, r, url, http.StatusMovedPermanently)
} else {
- writeJsonError(w, r, http.StatusNotFound, fmt.Errorf("volume id %d or collection %s not found", volumeId, collection))
+ writeJsonError(w, r, http.StatusNotFound, fmt.Errorf("volume id %s not found: %s", vid, location.Error))
}
}
@@ -123,13 +124,13 @@ func (ms *MasterServer) selfUrl(r *http.Request) string {
}
func (ms *MasterServer) submitFromMasterServerHandler(w http.ResponseWriter, r *http.Request) {
if ms.Topo.IsLeader() {
- submitForClientHandler(w, r, ms.selfUrl(r), ms.grpcDialOpiton)
+ submitForClientHandler(w, r, ms.selfUrl(r), ms.grpcDialOption)
} else {
masterUrl, err := ms.Topo.Leader()
if err != nil {
writeJsonError(w, r, http.StatusInternalServerError, err)
} else {
- submitForClientHandler(w, r, masterUrl, ms.grpcDialOpiton)
+ submitForClientHandler(w, r, masterUrl, ms.grpcDialOption)
}
}
}
@@ -144,7 +145,7 @@ func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGr
if replicationString == "" {
replicationString = ms.option.DefaultReplicaPlacement
}
- replicaPlacement, err := storage.NewReplicaPlacementFromString(replicationString)
+ replicaPlacement, err := super_block.NewReplicaPlacementFromString(replicationString)
if err != nil {
return nil, err
}
@@ -152,6 +153,11 @@ func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGr
if err != nil {
return nil, err
}
+ memoryMapMaxSizeMb, err := memory_map.ReadMemoryMapMaxSizeMb(r.FormValue("memoryMapMaxSizeMb"))
+ if err != nil {
+ return nil, err
+ }
+
preallocate := ms.preallocateSize
if r.FormValue("preallocate") != "" {
preallocate, err = strconv.ParseInt(r.FormValue("preallocate"), 10, 64)
@@ -160,13 +166,14 @@ func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGr
}
}
volumeGrowOption := &topology.VolumeGrowOption{
- Collection: r.FormValue("collection"),
- ReplicaPlacement: replicaPlacement,
- Ttl: ttl,
- Prealloacte: preallocate,
- DataCenter: r.FormValue("dataCenter"),
- Rack: r.FormValue("rack"),
- DataNode: r.FormValue("dataNode"),
+ Collection: r.FormValue("collection"),
+ ReplicaPlacement: replicaPlacement,
+ Ttl: ttl,
+ Prealloacte: preallocate,
+ DataCenter: r.FormValue("dataCenter"),
+ Rack: r.FormValue("rack"),
+ DataNode: r.FormValue("dataNode"),
+ MemoryMapMaxSizeMb: memoryMapMaxSizeMb,
}
return volumeGrowOption, nil
}
diff --git a/weed/server/raft_server.go b/weed/server/raft_server.go
index 88320ed98..53289f1c1 100644
--- a/weed/server/raft_server.go
+++ b/weed/server/raft_server.go
@@ -25,7 +25,7 @@ type RaftServer struct {
*raft.GrpcServer
}
-func NewRaftServer(grpcDialOption grpc.DialOption, peers []string, serverAddr string, dataDir string, topo *topology.Topology, pulseSeconds int) *RaftServer {
+func NewRaftServer(grpcDialOption grpc.DialOption, peers []string, serverAddr, dataDir string, topo *topology.Topology, pulseSeconds int) *RaftServer {
s := &RaftServer{
peers: peers,
serverAddr: serverAddr,
diff --git a/weed/server/volume_grpc_admin.go b/weed/server/volume_grpc_admin.go
index 35c2508a6..c631d2535 100644
--- a/weed/server/volume_grpc_admin.go
+++ b/weed/server/volume_grpc_admin.go
@@ -35,6 +35,7 @@ func (vs *VolumeServer) AllocateVolume(ctx context.Context, req *volume_server_p
req.Replication,
req.Ttl,
req.Preallocate,
+ req.MemoryMapMaxSizeMb,
)
if err != nil {
diff --git a/weed/server/volume_grpc_client_to_master.go b/weed/server/volume_grpc_client_to_master.go
index 731675b48..6038752d2 100644
--- a/weed/server/volume_grpc_client_to_master.go
+++ b/weed/server/volume_grpc_client_to_master.go
@@ -6,6 +6,7 @@ import (
"time"
"github.com/chrislusf/seaweedfs/weed/security"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/spf13/viper"
"google.golang.org/grpc"
@@ -90,6 +91,9 @@ func (vs *VolumeServer) doHeartbeat(ctx context.Context, masterNode, masterGrpcA
vs.MetricsAddress = in.GetMetricsAddress()
vs.MetricsIntervalSec = int(in.GetMetricsIntervalSeconds())
}
+ if len(in.StorageBackends) > 0 {
+ backend.LoadFromPbStorageBackends(in.StorageBackends)
+ }
}
}()
diff --git a/weed/server/volume_grpc_copy.go b/weed/server/volume_grpc_copy.go
index 8b39146ee..a54a1e343 100644
--- a/weed/server/volume_grpc_copy.go
+++ b/weed/server/volume_grpc_copy.go
@@ -55,11 +55,11 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
// println("source:", volFileInfoResp.String())
// copy ecx file
- if err := vs.doCopyFile(ctx, client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.IdxFileSize, volumeFileName, ".idx", false); err != nil {
+ if err := vs.doCopyFile(ctx, client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.IdxFileSize, volumeFileName, ".idx", false, false); err != nil {
return err
}
- if err := vs.doCopyFile(ctx, client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.DatFileSize, volumeFileName, ".dat", false); err != nil {
+ if err := vs.doCopyFile(ctx, client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.DatFileSize, volumeFileName, ".dat", false, true); err != nil {
return err
}
@@ -95,15 +95,16 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
}
func (vs *VolumeServer) doCopyFile(ctx context.Context, client volume_server_pb.VolumeServerClient, isEcVolume bool, collection string, vid uint32,
- compactRevision uint32, stopOffset uint64, baseFileName, ext string, isAppend bool) error {
+ compactRevision uint32, stopOffset uint64, baseFileName, ext string, isAppend bool, ignoreSourceFileNotFound bool) error {
copyFileClient, err := client.CopyFile(ctx, &volume_server_pb.CopyFileRequest{
- VolumeId: vid,
- Ext: ext,
- CompactionRevision: compactRevision,
- StopOffset: stopOffset,
- Collection: collection,
- IsEcVolume: isEcVolume,
+ VolumeId: vid,
+ Ext: ext,
+ CompactionRevision: compactRevision,
+ StopOffset: stopOffset,
+ Collection: collection,
+ IsEcVolume: isEcVolume,
+ IgnoreSourceFileNotFound: ignoreSourceFileNotFound,
})
if err != nil {
return fmt.Errorf("failed to start copying volume %d %s file: %v", vid, ext, err)
@@ -213,6 +214,9 @@ func (vs *VolumeServer) CopyFile(req *volume_server_pb.CopyFileRequest, stream v
}
}
if fileName == "" {
+ if req.IgnoreSourceFileNotFound {
+ return nil
+ }
return fmt.Errorf("CopyFile not found ec volume id %d", req.VolumeId)
}
}
@@ -221,6 +225,9 @@ func (vs *VolumeServer) CopyFile(req *volume_server_pb.CopyFileRequest, stream v
file, err := os.Open(fileName)
if err != nil {
+ if req.IgnoreSourceFileNotFound && err == os.ErrNotExist {
+ return nil
+ }
return err
}
defer file.Close()
@@ -257,7 +264,3 @@ func (vs *VolumeServer) CopyFile(req *volume_server_pb.CopyFileRequest, stream v
return nil
}
-
-func (vs *VolumeServer) findVolumeOrEcVolumeLocation(volumeId needle.VolumeId) {
-
-}
diff --git a/weed/server/volume_grpc_copy_incremental.go b/weed/server/volume_grpc_copy_incremental.go
index f56fbeef4..6d6c3daa3 100644
--- a/weed/server/volume_grpc_copy_incremental.go
+++ b/weed/server/volume_grpc_copy_incremental.go
@@ -4,9 +4,9 @@ import (
"context"
"fmt"
"io"
- "os"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
)
@@ -30,7 +30,7 @@ func (vs *VolumeServer) VolumeIncrementalCopy(req *volume_server_pb.VolumeIncrem
startOffset := foundOffset.ToAcutalOffset()
buf := make([]byte, 1024*1024*2)
- return sendFileContent(v.DataFile(), buf, startOffset, int64(stopOffset), stream)
+ return sendFileContent(v.DataBackend, buf, startOffset, int64(stopOffset), stream)
}
@@ -47,10 +47,10 @@ func (vs *VolumeServer) VolumeSyncStatus(ctx context.Context, req *volume_server
}
-func sendFileContent(datFile *os.File, buf []byte, startOffset, stopOffset int64, stream volume_server_pb.VolumeServer_VolumeIncrementalCopyServer) error {
+func sendFileContent(datBackend backend.BackendStorageFile, buf []byte, startOffset, stopOffset int64, stream volume_server_pb.VolumeServer_VolumeIncrementalCopyServer) error {
var blockSizeLimit = int64(len(buf))
for i := int64(0); i < stopOffset-startOffset; i += blockSizeLimit {
- n, readErr := datFile.ReadAt(buf, startOffset+i)
+ n, readErr := datBackend.ReadAt(buf, startOffset+i)
if readErr == nil || readErr == io.EOF {
resp := &volume_server_pb.VolumeIncrementalCopyResponse{}
resp.FileContent = buf[:int64(n)]
diff --git a/weed/server/volume_grpc_erasure_coding.go b/weed/server/volume_grpc_erasure_coding.go
index 8140a06f6..4bca9948e 100644
--- a/weed/server/volume_grpc_erasure_coding.go
+++ b/weed/server/volume_grpc_erasure_coding.go
@@ -8,10 +8,12 @@ import (
"math"
"os"
"path"
+ "path/filepath"
"strings"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
@@ -24,7 +26,7 @@ import (
Steps to apply erasure coding to .dat .idx files
0. ensure the volume is readonly
-1. client call VolumeEcShardsGenerate to generate the .ecx and .ec01~.ec14 files
+1. client call VolumeEcShardsGenerate to generate the .ecx and .ec00 ~ .ec13 files
2. client ask master for possible servers to hold the ec files, at least 4 servers
3. client call VolumeEcShardsCopy on above target servers to copy ec files from the source server
4. target servers report the new ec files to the master
@@ -33,7 +35,7 @@ Steps to apply erasure coding to .dat .idx files
*/
-// VolumeEcShardsGenerate generates the .ecx and .ec01 ~ .ec14 files
+// VolumeEcShardsGenerate generates the .ecx and .ec00 ~ .ec13 files
func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_server_pb.VolumeEcShardsGenerateRequest) (*volume_server_pb.VolumeEcShardsGenerateResponse, error) {
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
@@ -47,19 +49,24 @@ func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_
}
// write .ecx file
- if err := erasure_coding.WriteSortedEcxFile(baseFileName); err != nil {
- return nil, fmt.Errorf("WriteSortedEcxFile %s: %v", baseFileName, err)
+ if err := erasure_coding.WriteSortedFileFromIdx(baseFileName, ".ecx"); err != nil {
+ return nil, fmt.Errorf("WriteSortedFileFromIdx %s: %v", baseFileName, err)
}
- // write .ec01 ~ .ec14 files
+ // write .ec00 ~ .ec13 files
if err := erasure_coding.WriteEcFiles(baseFileName); err != nil {
return nil, fmt.Errorf("WriteEcFiles %s: %v", baseFileName, err)
}
+ // write .vif files
+ if err := pb.SaveVolumeInfo(baseFileName+".vif", &volume_server_pb.VolumeInfo{Version: uint32(v.Version())}); err != nil {
+ return nil, fmt.Errorf("WriteEcFiles %s: %v", baseFileName, err)
+ }
+
return &volume_server_pb.VolumeEcShardsGenerateResponse{}, nil
}
-// VolumeEcShardsRebuild generates the any of the missing .ec01 ~ .ec14 files
+// VolumeEcShardsRebuild generates the any of the missing .ec00 ~ .ec13 files
func (vs *VolumeServer) VolumeEcShardsRebuild(ctx context.Context, req *volume_server_pb.VolumeEcShardsRebuildRequest) (*volume_server_pb.VolumeEcShardsRebuildResponse, error) {
baseFileName := erasure_coding.EcShardBaseFileName(req.Collection, int(req.VolumeId))
@@ -68,7 +75,7 @@ func (vs *VolumeServer) VolumeEcShardsRebuild(ctx context.Context, req *volume_s
for _, location := range vs.store.Locations {
if util.FileExists(path.Join(location.Directory, baseFileName+".ecx")) {
- // write .ec01 ~ .ec14 files
+ // write .ec00 ~ .ec13 files
baseFileName = path.Join(location.Directory, baseFileName)
if generatedShardIds, err := erasure_coding.RebuildEcFiles(baseFileName); err != nil {
return nil, fmt.Errorf("RebuildEcFiles %s: %v", baseFileName, err)
@@ -103,23 +110,32 @@ func (vs *VolumeServer) VolumeEcShardsCopy(ctx context.Context, req *volume_serv
// copy ec data slices
for _, shardId := range req.ShardIds {
- if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, erasure_coding.ToExt(int(shardId)), false); err != nil {
+ if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, erasure_coding.ToExt(int(shardId)), false, false); err != nil {
return err
}
}
- if !req.CopyEcxFile {
+ if req.CopyEcxFile {
+
+ // copy ecx file
+ if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecx", false, false); err != nil {
+ return err
+ }
return nil
}
- // copy ecx file
- if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecx", false); err != nil {
- return err
+ if req.CopyEcjFile {
+ // copy ecj file
+ if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecj", true, true); err != nil {
+ return err
+ }
}
- // copy ecj file
- if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecj", true); err != nil {
- return err
+ if req.CopyVifFile {
+ // copy vif file
+ if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".vif", false, true); err != nil {
+ return err
+ }
}
return nil
@@ -137,6 +153,8 @@ func (vs *VolumeServer) VolumeEcShardsDelete(ctx context.Context, req *volume_se
baseFilename := erasure_coding.EcShardBaseFileName(req.Collection, int(req.VolumeId))
+ glog.V(0).Infof("ec volume %d shard delete %v", req.VolumeId, req.ShardIds)
+
found := false
for _, location := range vs.store.Locations {
if util.FileExists(path.Join(location.Directory, baseFilename+".ecx")) {
@@ -153,21 +171,22 @@ func (vs *VolumeServer) VolumeEcShardsDelete(ctx context.Context, req *volume_se
return nil, nil
}
- // check whether to delete the ecx file also
+ // check whether to delete the .ecx and .ecj file also
hasEcxFile := false
existingShardCount := 0
+ bName := filepath.Base(baseFilename)
for _, location := range vs.store.Locations {
fileInfos, err := ioutil.ReadDir(location.Directory)
if err != nil {
continue
}
for _, fileInfo := range fileInfos {
- if fileInfo.Name() == baseFilename+".ecx" {
+ if fileInfo.Name() == bName+".ecx" || fileInfo.Name() == bName+".ecj" {
hasEcxFile = true
continue
}
- if strings.HasPrefix(fileInfo.Name(), baseFilename+".ec") {
+ if strings.HasPrefix(fileInfo.Name(), bName+".ec") {
existingShardCount++
}
}
@@ -252,9 +271,14 @@ func (vs *VolumeServer) VolumeEcShardRead(req *volume_server_pb.VolumeEcShardRea
startOffset, bytesToRead := req.Offset, req.Size
for bytesToRead > 0 {
- bytesread, err := ecShard.ReadAt(buffer, startOffset)
+ // min of bytesToRead and bufSize
+ bufferSize := bufSize
+ if bufferSize > bytesToRead {
+ bufferSize = bytesToRead
+ }
+ bytesread, err := ecShard.ReadAt(buffer[0:bufferSize], startOffset)
- // println(fileName, "read", bytesread, "bytes, with target", bytesToRead)
+ // println("read", ecShard.FileName(), "startOffset", startOffset, bytesread, "bytes, with target", bufferSize)
if bytesread > 0 {
if int64(bytesread) > bytesToRead {
@@ -268,6 +292,7 @@ func (vs *VolumeServer) VolumeEcShardRead(req *volume_server_pb.VolumeEcShardRea
return err
}
+ startOffset += int64(bytesread)
bytesToRead -= int64(bytesread)
}
@@ -311,3 +336,35 @@ func (vs *VolumeServer) VolumeEcBlobDelete(ctx context.Context, req *volume_serv
return resp, nil
}
+
+// VolumeEcShardsToVolume generates the .idx, .dat files from .ecx, .ecj and .ec01 ~ .ec14 files
+func (vs *VolumeServer) VolumeEcShardsToVolume(ctx context.Context, req *volume_server_pb.VolumeEcShardsToVolumeRequest) (*volume_server_pb.VolumeEcShardsToVolumeResponse, error) {
+
+ v, found := vs.store.FindEcVolume(needle.VolumeId(req.VolumeId))
+ if !found {
+ return nil, fmt.Errorf("ec volume %d not found", req.VolumeId)
+ }
+ baseFileName := v.FileName()
+
+ if v.Collection != req.Collection {
+ return nil, fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
+ }
+
+ // calculate .dat file size
+ datFileSize, err := erasure_coding.FindDatFileSize(baseFileName)
+ if err != nil {
+ return nil, fmt.Errorf("FindDatFileSize %s: %v", baseFileName, err)
+ }
+
+ // write .dat file from .ec00 ~ .ec09 files
+ if err := erasure_coding.WriteDatFile(baseFileName, datFileSize); err != nil {
+ return nil, fmt.Errorf("WriteEcFiles %s: %v", baseFileName, err)
+ }
+
+ // write .idx file from .ecx and .ecj files
+ if err := erasure_coding.WriteIdxFileFromEcIndex(baseFileName); err != nil {
+ return nil, fmt.Errorf("WriteIdxFileFromEcIndex %s: %v", baseFileName, err)
+ }
+
+ return &volume_server_pb.VolumeEcShardsToVolumeResponse{}, nil
+}
diff --git a/weed/server/volume_grpc_query.go b/weed/server/volume_grpc_query.go
new file mode 100644
index 000000000..767e28e7b
--- /dev/null
+++ b/weed/server/volume_grpc_query.go
@@ -0,0 +1,69 @@
+package weed_server
+
+import (
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/query/json"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/tidwall/gjson"
+)
+
+func (vs *VolumeServer) Query(req *volume_server_pb.QueryRequest, stream volume_server_pb.VolumeServer_QueryServer) error {
+
+ for _, fid := range req.FromFileIds {
+
+ vid, id_cookie, err := operation.ParseFileId(fid)
+ if err != nil {
+ glog.V(0).Infof("volume query failed to parse fid %s: %v", fid, err)
+ return err
+ }
+
+ n := new(needle.Needle)
+ volumeId, _ := needle.NewVolumeId(vid)
+ n.ParsePath(id_cookie)
+
+ cookie := n.Cookie
+ if _, err := vs.store.ReadVolumeNeedle(volumeId, n); err != nil {
+ glog.V(0).Infof("volume query failed to read fid %s: %v", fid, err)
+ return err
+ }
+
+ if n.Cookie != cookie {
+ glog.V(0).Infof("volume query failed to read fid cookie %s: %v", fid, err)
+ return err
+ }
+
+ if req.InputSerialization.CsvInput != nil {
+
+ }
+
+ if req.InputSerialization.JsonInput != nil {
+
+ stripe := &volume_server_pb.QueriedStripe{
+ Records: nil,
+ }
+
+ filter := json.Query{
+ Field: req.Filter.Field,
+ Op: req.Filter.Operand,
+ Value: req.Filter.Value,
+ }
+ gjson.ForEachLine(string(n.Data), func(line gjson.Result) bool {
+ passedFilter, values := json.QueryJson(line.Raw, req.Selections, filter)
+ if !passedFilter {
+ return true
+ }
+ stripe.Records = json.ToJson(stripe.Records, req.Selections, values)
+ return true
+ })
+ err = stream.Send(stripe)
+ if err != nil {
+ return err
+ }
+ }
+
+ }
+
+ return nil
+}
diff --git a/weed/server/volume_grpc_tail.go b/weed/server/volume_grpc_tail.go
index 34c55a599..c26d6ed8f 100644
--- a/weed/server/volume_grpc_tail.go
+++ b/weed/server/volume_grpc_tail.go
@@ -10,6 +10,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
func (vs *VolumeServer) VolumeTailSender(req *volume_server_pb.VolumeTailSenderRequest, stream volume_server_pb.VolumeServer_VolumeTailSenderServer) error {
@@ -67,34 +68,13 @@ func sendNeedlesSince(stream volume_server_pb.VolumeServer_VolumeTailSenderServe
return lastTimestampNs, sendErr
}
- err = storage.ScanVolumeFileNeedleFrom(v.Version(), v.DataFile(), foundOffset.ToAcutalOffset(), func(needleHeader, needleBody []byte, needleAppendAtNs uint64) error {
-
- isLastChunk := false
-
- // need to send body by chunks
- for i := 0; i < len(needleBody); i += BufferSizeLimit {
- stopOffset := i + BufferSizeLimit
- if stopOffset >= len(needleBody) {
- isLastChunk = true
- stopOffset = len(needleBody)
- }
-
- sendErr := stream.Send(&volume_server_pb.VolumeTailSenderResponse{
- NeedleHeader: needleHeader,
- NeedleBody: needleBody[i:stopOffset],
- IsLastChunk: isLastChunk,
- })
- if sendErr != nil {
- return sendErr
- }
- }
-
- lastProcessedTimestampNs = needleAppendAtNs
- return nil
+ scanner := &VolumeFileScanner4Tailing{
+ stream: stream,
+ }
- })
+ err = storage.ScanVolumeFileFrom(v.Version(), v.DataBackend, foundOffset.ToAcutalOffset(), scanner)
- return
+ return scanner.lastProcessedTimestampNs, err
}
@@ -115,3 +95,42 @@ func (vs *VolumeServer) VolumeTailReceiver(ctx context.Context, req *volume_serv
})
}
+
+// generate the volume idx
+type VolumeFileScanner4Tailing struct {
+ stream volume_server_pb.VolumeServer_VolumeTailSenderServer
+ lastProcessedTimestampNs uint64
+}
+
+func (scanner *VolumeFileScanner4Tailing) VisitSuperBlock(superBlock super_block.SuperBlock) error {
+ return nil
+
+}
+func (scanner *VolumeFileScanner4Tailing) ReadNeedleBody() bool {
+ return true
+}
+
+func (scanner *VolumeFileScanner4Tailing) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
+ isLastChunk := false
+
+ // need to send body by chunks
+ for i := 0; i < len(needleBody); i += BufferSizeLimit {
+ stopOffset := i + BufferSizeLimit
+ if stopOffset >= len(needleBody) {
+ isLastChunk = true
+ stopOffset = len(needleBody)
+ }
+
+ sendErr := scanner.stream.Send(&volume_server_pb.VolumeTailSenderResponse{
+ NeedleHeader: needleHeader,
+ NeedleBody: needleBody[i:stopOffset],
+ IsLastChunk: isLastChunk,
+ })
+ if sendErr != nil {
+ return sendErr
+ }
+ }
+
+ scanner.lastProcessedTimestampNs = n.AppendAtNs
+ return nil
+}
diff --git a/weed/server/volume_grpc_tier_download.go b/weed/server/volume_grpc_tier_download.go
new file mode 100644
index 000000000..7b3982e40
--- /dev/null
+++ b/weed/server/volume_grpc_tier_download.go
@@ -0,0 +1,85 @@
+package weed_server
+
+import (
+ "fmt"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+// VolumeTierMoveDatFromRemote copy dat file from a remote tier to local volume server
+func (vs *VolumeServer) VolumeTierMoveDatFromRemote(req *volume_server_pb.VolumeTierMoveDatFromRemoteRequest, stream volume_server_pb.VolumeServer_VolumeTierMoveDatFromRemoteServer) error {
+
+ // find existing volume
+ v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
+ if v == nil {
+ return fmt.Errorf("volume %d not found", req.VolumeId)
+ }
+
+ // verify the collection
+ if v.Collection != req.Collection {
+ return fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
+ }
+
+ // locate the disk file
+ storageName, storageKey := v.RemoteStorageNameKey()
+ if storageName == "" || storageKey == "" {
+ return fmt.Errorf("volume %d is already on local disk", req.VolumeId)
+ }
+
+ // check whether the local .dat already exists
+ _, ok := v.DataBackend.(*backend.DiskFile)
+ if ok {
+ return fmt.Errorf("volume %d is already on local disk", req.VolumeId)
+ }
+
+ // check valid storage backend type
+ backendStorage, found := backend.BackendStorages[storageName]
+ if !found {
+ var keys []string
+ for key := range backend.BackendStorages {
+ keys = append(keys, key)
+ }
+ return fmt.Errorf("remote storage %s not found from suppported: %v", storageName, keys)
+ }
+
+ startTime := time.Now()
+ fn := func(progressed int64, percentage float32) error {
+ now := time.Now()
+ if now.Sub(startTime) < time.Second {
+ return nil
+ }
+ startTime = now
+ return stream.Send(&volume_server_pb.VolumeTierMoveDatFromRemoteResponse{
+ Processed: progressed,
+ ProcessedPercentage: percentage,
+ })
+ }
+ // copy the data file
+ _, err := backendStorage.DownloadFile(v.FileName()+".dat", storageKey, fn)
+ if err != nil {
+ return fmt.Errorf("backend %s copy file %s: %v", storageName, v.FileName()+".dat", err)
+ }
+
+ if req.KeepRemoteDatFile {
+ return nil
+ }
+
+ // remove remote file
+ if err := backendStorage.DeleteFile(storageKey); err != nil {
+ return fmt.Errorf("volume %d fail to delete remote file %s: %v", v.Id, storageKey, err)
+ }
+
+ // forget remote file
+ v.GetVolumeInfo().Files = v.GetVolumeInfo().Files[1:]
+ if err := v.SaveVolumeInfo(); err != nil {
+ return fmt.Errorf("volume %d fail to save remote file info: %v", v.Id, err)
+ }
+
+ v.DataBackend.Close()
+ v.DataBackend = nil
+
+ return nil
+}
diff --git a/weed/server/volume_grpc_tier_upload.go b/weed/server/volume_grpc_tier_upload.go
new file mode 100644
index 000000000..c9694df59
--- /dev/null
+++ b/weed/server/volume_grpc_tier_upload.go
@@ -0,0 +1,100 @@
+package weed_server
+
+import (
+ "fmt"
+ "os"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+// VolumeTierMoveDatToRemote copy dat file to a remote tier
+func (vs *VolumeServer) VolumeTierMoveDatToRemote(req *volume_server_pb.VolumeTierMoveDatToRemoteRequest, stream volume_server_pb.VolumeServer_VolumeTierMoveDatToRemoteServer) error {
+
+ // find existing volume
+ v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
+ if v == nil {
+ return fmt.Errorf("volume %d not found", req.VolumeId)
+ }
+
+ // verify the collection
+ if v.Collection != req.Collection {
+ return fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
+ }
+
+ // locate the disk file
+ diskFile, ok := v.DataBackend.(*backend.DiskFile)
+ if !ok {
+ return fmt.Errorf("volume %d is not on local disk", req.VolumeId)
+ }
+
+ // check valid storage backend type
+ backendStorage, found := backend.BackendStorages[req.DestinationBackendName]
+ if !found {
+ var keys []string
+ for key := range backend.BackendStorages {
+ keys = append(keys, key)
+ }
+ return fmt.Errorf("destination %s not found, suppported: %v", req.DestinationBackendName, keys)
+ }
+
+ // check whether the existing backend storage is the same as requested
+ // if same, skip
+ backendType, backendId := backend.BackendNameToTypeId(req.DestinationBackendName)
+ for _, remoteFile := range v.GetVolumeInfo().GetFiles() {
+ if remoteFile.BackendType == backendType && remoteFile.BackendId == backendId {
+ return fmt.Errorf("destination %s already exists", req.DestinationBackendName)
+ }
+ }
+
+ startTime := time.Now()
+ fn := func(progressed int64, percentage float32) error {
+ now := time.Now()
+ if now.Sub(startTime) < time.Second {
+ return nil
+ }
+ startTime = now
+ return stream.Send(&volume_server_pb.VolumeTierMoveDatToRemoteResponse{
+ Processed: progressed,
+ ProcessedPercentage: percentage,
+ })
+ }
+
+ // remember the file original source
+ attributes := make(map[string]string)
+ attributes["volumeId"] = v.Id.String()
+ attributes["collection"] = v.Collection
+ attributes["ext"] = ".dat"
+ // copy the data file
+ key, size, err := backendStorage.CopyFile(diskFile.File, attributes, fn)
+ if err != nil {
+ return fmt.Errorf("backend %s copy file %s: %v", req.DestinationBackendName, diskFile.Name(), err)
+ }
+
+ // save the remote file to volume tier info
+ v.GetVolumeInfo().Files = append(v.GetVolumeInfo().GetFiles(), &volume_server_pb.RemoteFile{
+ BackendType: backendType,
+ BackendId: backendId,
+ Key: key,
+ Offset: 0,
+ FileSize: uint64(size),
+ ModifiedTime: uint64(time.Now().Unix()),
+ Extension: ".dat",
+ })
+
+ if err := v.SaveVolumeInfo(); err != nil {
+ return fmt.Errorf("volume %d fail to save remote file info: %v", v.Id, err)
+ }
+
+ if err := v.LoadRemoteFile(); err != nil {
+ return fmt.Errorf("volume %d fail to load remote file: %v", v.Id, err)
+ }
+
+ if !req.KeepLocalDatFile {
+ os.Remove(v.FileName() + ".dat")
+ }
+
+ return nil
+}
diff --git a/weed/server/volume_server_handlers_admin.go b/weed/server/volume_server_handlers_admin.go
index 25b6582f7..1938a34c4 100644
--- a/weed/server/volume_server_handlers_admin.go
+++ b/weed/server/volume_server_handlers_admin.go
@@ -12,7 +12,7 @@ import (
func (vs *VolumeServer) statusHandler(w http.ResponseWriter, r *http.Request) {
m := make(map[string]interface{})
m["Version"] = util.VERSION
- m["Volumes"] = vs.store.Status()
+ m["Volumes"] = vs.store.VolumeInfos()
writeJsonQuiet(w, r, http.StatusOK, m)
}
diff --git a/weed/server/volume_server_handlers_read.go b/weed/server/volume_server_handlers_read.go
index f30ffefaf..cd11356b9 100644
--- a/weed/server/volume_server_handlers_read.go
+++ b/weed/server/volume_server_handlers_read.go
@@ -4,6 +4,7 @@ import (
"bytes"
"context"
"errors"
+ "fmt"
"io"
"mime"
"mime/multipart"
@@ -66,7 +67,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
glog.V(2).Infoln("volume", volumeId, "found on", lookupResult, "error", err)
if err == nil && len(lookupResult.Locations) > 0 {
u, _ := url.Parse(util.NormalizeUrl(lookupResult.Locations[0].PublicUrl))
- u.Path = r.URL.Path
+ u.Path = fmt.Sprintf("%s/%s,%s", u.Path, vid, fid)
arg := url.Values{}
if c := r.FormValue("collection"); c != "" {
arg.Set("collection", c)
diff --git a/weed/server/volume_server_handlers_ui.go b/weed/server/volume_server_handlers_ui.go
index 852f0b751..8d35c9c8b 100644
--- a/weed/server/volume_server_handlers_ui.go
+++ b/weed/server/volume_server_handlers_ui.go
@@ -8,6 +8,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
ui "github.com/chrislusf/seaweedfs/weed/server/volume_server_ui"
"github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -20,19 +21,30 @@ func (vs *VolumeServer) uiStatusHandler(w http.ResponseWriter, r *http.Request)
ds = append(ds, stats.NewDiskStatus(dir))
}
}
+ volumeInfos := vs.store.VolumeInfos()
+ var normalVolumeInfos, remoteVolumeInfos []*storage.VolumeInfo
+ for _, vinfo := range volumeInfos {
+ if vinfo.IsRemote() {
+ remoteVolumeInfos = append(remoteVolumeInfos, vinfo)
+ } else {
+ normalVolumeInfos = append(normalVolumeInfos, vinfo)
+ }
+ }
args := struct {
- Version string
- Masters []string
- Volumes interface{}
- EcVolumes interface{}
- DiskStatuses interface{}
- Stats interface{}
- Counters *stats.ServerStats
+ Version string
+ Masters []string
+ Volumes interface{}
+ EcVolumes interface{}
+ RemoteVolumes interface{}
+ DiskStatuses interface{}
+ Stats interface{}
+ Counters *stats.ServerStats
}{
util.VERSION,
vs.SeedMasterNodes,
- vs.store.Status(),
+ normalVolumeInfos,
vs.store.EcVolumes(),
+ remoteVolumeInfos,
ds,
infos,
serverStats,
diff --git a/weed/server/volume_server_handlers_write.go b/weed/server/volume_server_handlers_write.go
index db8fcb555..05e21612b 100644
--- a/weed/server/volume_server_handlers_write.go
+++ b/weed/server/volume_server_handlers_write.go
@@ -51,10 +51,14 @@ func (vs *VolumeServer) PostHandler(w http.ResponseWriter, r *http.Request) {
ret := operation.UploadResult{}
_, isUnchanged, writeError := topology.ReplicatedWrite(vs.GetMaster(), vs.store, volumeId, needle, r)
- httpStatus := http.StatusCreated
- if isUnchanged {
- httpStatus = http.StatusNotModified
+
+ // http 304 status code does not allow body
+ if writeError == nil && isUnchanged {
+ w.WriteHeader(http.StatusNotModified)
+ return
}
+
+ httpStatus := http.StatusCreated
if writeError != nil {
httpStatus = http.StatusInternalServerError
ret.Error = writeError.Error()
diff --git a/weed/server/volume_server_ui/templates.go b/weed/server/volume_server_ui/templates.go
index eafc0aaeb..81496b1de 100644
--- a/weed/server/volume_server_ui/templates.go
+++ b/weed/server/volume_server_ui/templates.go
@@ -107,10 +107,11 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`
Id
Collection
-
Size
+
Data Size
Files
Trash
TTL
+
ReadOnly
@@ -122,6 +123,37 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`{{ .FileCount }}
{{ .DeleteCount }} / {{.DeletedByteCount}} Bytes
{{ .Ttl }}
+ {{ .ReadOnly }}
+
+ {{ end }}
+
+
+
+
+
+
Remote Volumes
+
+
+
+ Id
+ Collection
+ Size
+ Files
+ Trash
+ Remote
+ Key
+
+
+
+ {{ range .RemoteVolumes }}
+
+ {{ .Id }}
+ {{ .Collection }}
+ {{ .Size }} Bytes
+ {{ .FileCount }}
+ {{ .DeleteCount }} / {{.DeletedByteCount}} Bytes
+ {{ .RemoteStorageName }}
+ {{ .RemoteStorageKey }}
{{ end }}
diff --git a/weed/server/webdav_server.go b/weed/server/webdav_server.go
index 151b48a78..abd0b66eb 100644
--- a/weed/server/webdav_server.go
+++ b/weed/server/webdav_server.go
@@ -10,16 +10,18 @@ import (
"strings"
"time"
+ "golang.org/x/net/webdav"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "golang.org/x/net/webdav"
- "google.golang.org/grpc"
+
+ "github.com/spf13/viper"
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/security"
- "github.com/spf13/viper"
)
type WebDavOption struct {
@@ -163,7 +165,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
func (fs *WebDavFileSystem) OpenFile(ctx context.Context, fullFilePath string, flag int, perm os.FileMode) (webdav.File, error) {
- glog.V(2).Infof("WebDavFileSystem.OpenFile %v", fullFilePath)
+ glog.V(2).Infof("WebDavFileSystem.OpenFile %v %x", fullFilePath, flag)
var err error
if fullFilePath, err = clearName(fullFilePath); err != nil {
@@ -175,12 +177,6 @@ func (fs *WebDavFileSystem) OpenFile(ctx context.Context, fullFilePath string, f
if strings.HasSuffix(fullFilePath, "/") {
return nil, os.ErrInvalid
}
- // based directory should be exists.
- dir, _ := path.Split(fullFilePath)
- _, err := fs.stat(ctx, dir)
- if err != nil {
- return nil, os.ErrInvalid
- }
_, err = fs.stat(ctx, fullFilePath)
if err == nil {
if flag&os.O_EXCL != 0 {
@@ -412,7 +408,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
fileUrl := fmt.Sprintf("http://%s/%s", host, fileId)
bufReader := bytes.NewReader(buf)
- uploadResult, err := operation.Upload(fileUrl, f.name, bufReader, false, "application/octet-stream", nil, auth)
+ uploadResult, err := operation.Upload(fileUrl, f.name, bufReader, false, "", nil, auth)
if err != nil {
glog.V(0).Infof("upload data %v to %s: %v", f.name, fileUrl, err)
return 0, fmt.Errorf("upload data: %v", err)
@@ -448,9 +444,11 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
return nil
})
- if err != nil {
+ if err == nil {
+ glog.V(3).Infof("WebDavFileSystem.Write %v: written [%d,%d)", f.name, f.off, f.off+int64(len(buf)))
f.off += int64(len(buf))
}
+
return len(buf), err
}
@@ -494,10 +492,13 @@ func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
}
readSize = int(totalRead)
+ glog.V(3).Infof("WebDavFileSystem.Read %v: [%d,%d)", f.name, f.off, f.off+totalRead)
+
f.off += totalRead
if readSize == 0 {
return 0, io.EOF
}
+
return
}
@@ -511,7 +512,7 @@ func (f *WebDavFile) Readdir(count int) (ret []os.FileInfo, err error) {
dir = dir[:len(dir)-1]
}
- err = filer2.ReadDirAllEntries(ctx, f.fs, dir, func(entry *filer_pb.Entry) {
+ err = filer2.ReadDirAllEntries(ctx, f.fs, dir, "", func(entry *filer_pb.Entry, isLast bool) {
fi := FileInfo{
size: int64(filer2.TotalSize(entry.GetChunks())),
name: entry.Name,
diff --git a/weed/shell/command_ec_balance.go b/weed/shell/command_ec_balance.go
index 47ae7bad3..96599372e 100644
--- a/weed/shell/command_ec_balance.go
+++ b/weed/shell/command_ec_balance.go
@@ -207,7 +207,7 @@ func doDeduplicateEcShards(ctx context.Context, commandEnv *CommandEnv, collecti
if len(ecNodes) <= 1 {
continue
}
- sortEcNodes(ecNodes)
+ sortEcNodesByFreeslotsAscending(ecNodes)
fmt.Printf("ec shard %d.%d has %d copies, keeping %v\n", vid, shardId, len(ecNodes), ecNodes[0].info.Id)
if !applyBalancing {
continue
@@ -266,6 +266,10 @@ func doBalanceEcShardsAcrossRacks(ctx context.Context, commandEnv *CommandEnv, c
for shardId, ecNode := range ecShardsToMove {
rackId := pickOneRack(racks, rackToShardCount, averageShardsPerEcRack)
+ if rackId == "" {
+ fmt.Printf("ec shard %d.%d at %s can not find a destination rack\n", vid, shardId, ecNode.info.Id)
+ continue
+ }
var possibleDestinationEcNodes []*EcNode
for _, n := range racks[rackId].ecNodes {
possibleDestinationEcNodes = append(possibleDestinationEcNodes, n)
@@ -436,10 +440,9 @@ func doBalanceEcRack(ctx context.Context, commandEnv *CommandEnv, ecRack *EcRack
return nil
}
-func pickOneEcNodeAndMoveOneShard(ctx context.Context, commandEnv *CommandEnv, expectedTotalEcShards int, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
+func pickOneEcNodeAndMoveOneShard(ctx context.Context, commandEnv *CommandEnv, averageShardsPerEcNode int, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
- sortEcNodes(possibleDestinationEcNodes)
- averageShardsPerEcNode := ceilDivide(expectedTotalEcShards, len(possibleDestinationEcNodes))
+ sortEcNodesByFreeslotsDecending(possibleDestinationEcNodes)
for _, destEcNode := range possibleDestinationEcNodes {
if destEcNode.info.Id == existingLocation.info.Id {
diff --git a/weed/shell/command_ec_common.go b/weed/shell/command_ec_common.go
index d0fe16a68..2beed4742 100644
--- a/weed/shell/command_ec_common.go
+++ b/weed/shell/command_ec_common.go
@@ -22,7 +22,7 @@ func moveMountedShardToEcNode(ctx context.Context, commandEnv *CommandEnv, exist
if applyBalancing {
// ask destination node to copy shard and the ecx file from source node, and mount it
- copiedShardIds, err = oneServerCopyAndMountEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, destinationEcNode, uint32(shardId), 1, vid, collection, existingLocation.info.Id)
+ copiedShardIds, err = oneServerCopyAndMountEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, destinationEcNode, []uint32{uint32(shardId)}, vid, collection, existingLocation.info.Id)
if err != nil {
return err
}
@@ -51,13 +51,9 @@ func moveMountedShardToEcNode(ctx context.Context, commandEnv *CommandEnv, exist
}
func oneServerCopyAndMountEcShardsFromSource(ctx context.Context, grpcDialOption grpc.DialOption,
- targetServer *EcNode, startFromShardId uint32, shardCount int,
+ targetServer *EcNode, shardIdsToCopy []uint32,
volumeId needle.VolumeId, collection string, existingLocation string) (copiedShardIds []uint32, err error) {
- var shardIdsToCopy []uint32
- for shardId := startFromShardId; shardId < startFromShardId+uint32(shardCount); shardId++ {
- shardIdsToCopy = append(shardIdsToCopy, shardId)
- }
fmt.Printf("allocate %d.%v %s => %s\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id)
err = operation.WithVolumeServerClient(targetServer.info.Id, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
@@ -70,6 +66,8 @@ func oneServerCopyAndMountEcShardsFromSource(ctx context.Context, grpcDialOption
Collection: collection,
ShardIds: shardIdsToCopy,
CopyEcxFile: true,
+ CopyEcjFile: true,
+ CopyVifFile: true,
SourceDataNode: existingLocation,
})
if copyErr != nil {
@@ -112,12 +110,18 @@ func eachDataNode(topo *master_pb.TopologyInfo, fn func(dc string, rack RackId,
}
}
-func sortEcNodes(ecNodes []*EcNode) {
+func sortEcNodesByFreeslotsDecending(ecNodes []*EcNode) {
sort.Slice(ecNodes, func(i, j int) bool {
return ecNodes[i].freeEcSlot > ecNodes[j].freeEcSlot
})
}
+func sortEcNodesByFreeslotsAscending(ecNodes []*EcNode) {
+ sort.Slice(ecNodes, func(i, j int) bool {
+ return ecNodes[i].freeEcSlot < ecNodes[j].freeEcSlot
+ })
+}
+
type CandidateEcNode struct {
ecNode *EcNode
shardCount int
@@ -156,7 +160,7 @@ func countShards(ecShardInfos []*master_pb.VolumeEcShardInformationMessage) (cou
}
func countFreeShardSlots(dn *master_pb.DataNodeInfo) (count int) {
- return int(dn.FreeVolumeCount)*10 - countShards(dn.EcShardInfos)
+ return int(dn.MaxVolumeCount-dn.ActiveVolumeCount)*erasure_coding.DataShardsCount - countShards(dn.EcShardInfos)
}
type RackId string
@@ -191,18 +195,18 @@ func collectEcNodes(ctx context.Context, commandEnv *CommandEnv, selectedDataCen
if selectedDataCenter != "" && selectedDataCenter != dc {
return
}
- if freeEcSlots := countFreeShardSlots(dn); freeEcSlots > 0 {
- ecNodes = append(ecNodes, &EcNode{
- info: dn,
- dc: dc,
- rack: rack,
- freeEcSlot: int(freeEcSlots),
- })
- totalFreeEcSlots += freeEcSlots
- }
+
+ freeEcSlots := countFreeShardSlots(dn)
+ ecNodes = append(ecNodes, &EcNode{
+ info: dn,
+ dc: dc,
+ rack: rack,
+ freeEcSlot: int(freeEcSlots),
+ })
+ totalFreeEcSlots += freeEcSlots
})
- sortEcNodes(ecNodes)
+ sortEcNodesByFreeslotsDecending(ecNodes)
return
}
diff --git a/weed/shell/command_ec_decode.go b/weed/shell/command_ec_decode.go
new file mode 100644
index 000000000..1f9ad2ff9
--- /dev/null
+++ b/weed/shell/command_ec_decode.go
@@ -0,0 +1,265 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+func init() {
+ Commands = append(Commands, &commandEcDecode{})
+}
+
+type commandEcDecode struct {
+}
+
+func (c *commandEcDecode) Name() string {
+ return "ec.decode"
+}
+
+func (c *commandEcDecode) Help() string {
+ return `decode a erasure coded volume into a normal volume
+
+ ec.decode [-collection=""] [-volumeId=]
+
+`
+}
+
+func (c *commandEcDecode) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ encodeCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ volumeId := encodeCommand.Int("volumeId", 0, "the volume id")
+ collection := encodeCommand.String("collection", "", "the collection name")
+ if err = encodeCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ ctx := context.Background()
+ vid := needle.VolumeId(*volumeId)
+
+ // collect topology information
+ topologyInfo, err := collectTopologyInfo(ctx, commandEnv)
+ if err != nil {
+ return err
+ }
+
+ // volumeId is provided
+ if vid != 0 {
+ return doEcDecode(ctx, commandEnv, topologyInfo, *collection, vid)
+ }
+
+ // apply to all volumes in the collection
+ volumeIds := collectEcShardIds(topologyInfo, *collection)
+ fmt.Printf("ec encode volumes: %v\n", volumeIds)
+ for _, vid := range volumeIds {
+ if err = doEcDecode(ctx, commandEnv, topologyInfo, *collection, vid); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
+func doEcDecode(ctx context.Context, commandEnv *CommandEnv, topoInfo *master_pb.TopologyInfo, collection string, vid needle.VolumeId) (err error) {
+ // find volume location
+ nodeToEcIndexBits := collectEcNodeShardBits(topoInfo, vid)
+
+ fmt.Printf("ec volume %d shard locations: %+v\n", vid, nodeToEcIndexBits)
+
+ // collect ec shards to the server with most space
+ targetNodeLocation, err := collectEcShards(ctx, commandEnv, nodeToEcIndexBits, collection, vid)
+ if err != nil {
+ return fmt.Errorf("collectEcShards for volume %d: %v", vid, err)
+ }
+
+ // generate a normal volume
+ err = generateNormalVolume(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, targetNodeLocation)
+ if err != nil {
+ return fmt.Errorf("generate normal volume %d on %s: %v", vid, targetNodeLocation, err)
+ }
+
+ // delete the previous ec shards
+ err = mountVolumeAndDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, targetNodeLocation, nodeToEcIndexBits, vid)
+ if err != nil {
+ return fmt.Errorf("delete ec shards for volume %d: %v", vid, err)
+ }
+
+ return nil
+}
+
+func mountVolumeAndDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialOption, collection, targetNodeLocation string, nodeToEcIndexBits map[string]erasure_coding.ShardBits, vid needle.VolumeId) error {
+
+ // mount volume
+ if err := operation.WithVolumeServerClient(targetNodeLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ _, mountErr := volumeServerClient.VolumeMount(ctx, &volume_server_pb.VolumeMountRequest{
+ VolumeId: uint32(vid),
+ })
+ return mountErr
+ }); err != nil {
+ return fmt.Errorf("mountVolumeAndDeleteEcShards mount volume %d on %s: %v", vid, targetNodeLocation, err)
+ }
+
+ // unmount ec shards
+ for location, ecIndexBits := range nodeToEcIndexBits {
+ fmt.Printf("unmount ec volume %d on %s has shards: %+v\n", vid, location, ecIndexBits.ShardIds())
+ err := unmountEcShards(ctx, grpcDialOption, vid, location, ecIndexBits.ToUint32Slice())
+ if err != nil {
+ return fmt.Errorf("mountVolumeAndDeleteEcShards unmount ec volume %d on %s: %v", vid, location, err)
+ }
+ }
+ // delete ec shards
+ for location, ecIndexBits := range nodeToEcIndexBits {
+ fmt.Printf("delete ec volume %d on %s has shards: %+v\n", vid, location, ecIndexBits.ShardIds())
+ err := sourceServerDeleteEcShards(ctx, grpcDialOption, collection, vid, location, ecIndexBits.ToUint32Slice())
+ if err != nil {
+ return fmt.Errorf("mountVolumeAndDeleteEcShards delete ec volume %d on %s: %v", vid, location, err)
+ }
+ }
+
+ return nil
+}
+
+func generateNormalVolume(ctx context.Context, grpcDialOption grpc.DialOption, vid needle.VolumeId, collection string, sourceVolumeServer string) error {
+
+ fmt.Printf("generateNormalVolume from ec volume %d on %s\n", vid, sourceVolumeServer)
+
+ err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ _, genErr := volumeServerClient.VolumeEcShardsToVolume(ctx, &volume_server_pb.VolumeEcShardsToVolumeRequest{
+ VolumeId: uint32(vid),
+ Collection: collection,
+ })
+ return genErr
+ })
+
+ return err
+
+}
+
+func collectEcShards(ctx context.Context, commandEnv *CommandEnv, nodeToEcIndexBits map[string]erasure_coding.ShardBits, collection string, vid needle.VolumeId) (targetNodeLocation string, err error) {
+
+ maxShardCount := 0
+ var exisitngEcIndexBits erasure_coding.ShardBits
+ for loc, ecIndexBits := range nodeToEcIndexBits {
+ toBeCopiedShardCount := ecIndexBits.MinusParityShards().ShardIdCount()
+ if toBeCopiedShardCount > maxShardCount {
+ maxShardCount = toBeCopiedShardCount
+ targetNodeLocation = loc
+ exisitngEcIndexBits = ecIndexBits
+ }
+ }
+
+ fmt.Printf("collectEcShards: ec volume %d collect shards to %s from: %+v\n", vid, targetNodeLocation, nodeToEcIndexBits)
+
+ var copiedEcIndexBits erasure_coding.ShardBits
+ for loc, ecIndexBits := range nodeToEcIndexBits {
+ if loc == targetNodeLocation {
+ continue
+ }
+
+ needToCopyEcIndexBits := ecIndexBits.Minus(exisitngEcIndexBits).MinusParityShards()
+ if needToCopyEcIndexBits.ShardIdCount() == 0 {
+ continue
+ }
+
+ err = operation.WithVolumeServerClient(targetNodeLocation, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+
+ fmt.Printf("copy %d.%v %s => %s\n", vid, needToCopyEcIndexBits.ShardIds(), loc, targetNodeLocation)
+
+ _, copyErr := volumeServerClient.VolumeEcShardsCopy(ctx, &volume_server_pb.VolumeEcShardsCopyRequest{
+ VolumeId: uint32(vid),
+ Collection: collection,
+ ShardIds: needToCopyEcIndexBits.ToUint32Slice(),
+ CopyEcxFile: false,
+ CopyEcjFile: true,
+ CopyVifFile: true,
+ SourceDataNode: loc,
+ })
+ if copyErr != nil {
+ return fmt.Errorf("copy %d.%v %s => %s : %v\n", vid, needToCopyEcIndexBits.ShardIds(), loc, targetNodeLocation, copyErr)
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ break
+ }
+
+ copiedEcIndexBits = copiedEcIndexBits.Plus(needToCopyEcIndexBits)
+
+ }
+
+ nodeToEcIndexBits[targetNodeLocation] = exisitngEcIndexBits.Plus(copiedEcIndexBits)
+
+ return targetNodeLocation, err
+
+}
+
+func collectTopologyInfo(ctx context.Context, commandEnv *CommandEnv) (topoInfo *master_pb.TopologyInfo, err error) {
+
+ var resp *master_pb.VolumeListResponse
+ err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
+ return err
+ })
+ if err != nil {
+ return
+ }
+
+ return resp.TopologyInfo, nil
+
+}
+
+func collectEcShardInfos(topoInfo *master_pb.TopologyInfo, selectedCollection string, vid needle.VolumeId) (ecShardInfos []*master_pb.VolumeEcShardInformationMessage) {
+
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.EcShardInfos {
+ if v.Collection == selectedCollection && v.Id == uint32(vid) {
+ ecShardInfos = append(ecShardInfos, v)
+ }
+ }
+ })
+
+ return
+}
+
+func collectEcShardIds(topoInfo *master_pb.TopologyInfo, selectedCollection string) (vids []needle.VolumeId) {
+
+ vidMap := make(map[uint32]bool)
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.EcShardInfos {
+ if v.Collection == selectedCollection {
+ vidMap[v.Id] = true
+ }
+ }
+ })
+
+ for vid := range vidMap {
+ vids = append(vids, needle.VolumeId(vid))
+ }
+
+ return
+}
+
+func collectEcNodeShardBits(topoInfo *master_pb.TopologyInfo, vid needle.VolumeId) map[string]erasure_coding.ShardBits {
+
+ nodeToEcIndexBits := make(map[string]erasure_coding.ShardBits)
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.EcShardInfos {
+ if v.Id == uint32(vid) {
+ nodeToEcIndexBits[dn.Id] = erasure_coding.ShardBits(v.EcIndexBits)
+ }
+ }
+ })
+
+ return nodeToEcIndexBits
+}
diff --git a/weed/shell/command_ec_encode.go b/weed/shell/command_ec_encode.go
index 8ad0d51c8..58527abf2 100644
--- a/weed/shell/command_ec_encode.go
+++ b/weed/shell/command_ec_encode.go
@@ -8,13 +8,14 @@ import (
"sync"
"time"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/wdclient"
- "google.golang.org/grpc"
)
func init() {
@@ -87,15 +88,17 @@ func (c *commandEcEncode) Do(args []string, commandEnv *CommandEnv, writer io.Wr
func doEcEncode(ctx context.Context, commandEnv *CommandEnv, collection string, vid needle.VolumeId) (err error) {
// find volume location
- locations := commandEnv.MasterClient.GetLocations(uint32(vid))
- if len(locations) == 0 {
+ locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
+ if !found {
return fmt.Errorf("volume %d not found", vid)
}
+ // fmt.Printf("found ec %d shards on %v\n", vid, locations)
+
// mark the volume as readonly
err = markVolumeReadonly(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
if err != nil {
- return fmt.Errorf("generate ec shards for volume %d on %s: %v", vid, locations[0].Url, err)
+ return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
}
// generate ec shards
@@ -163,10 +166,10 @@ func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle
}
// calculate how many shards to allocate for these servers
- allocated := balancedEcDistribution(allocatedDataNodes)
+ allocatedEcIds := balancedEcDistribution(allocatedDataNodes)
// ask the data nodes to copy from the source volume server
- copiedShardIds, err := parallelCopyEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, allocatedDataNodes, allocated, volumeId, collection, existingLocations[0])
+ copiedShardIds, err := parallelCopyEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, allocatedDataNodes, allocatedEcIds, volumeId, collection, existingLocations[0])
if err != nil {
return err
}
@@ -196,31 +199,29 @@ func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle
}
func parallelCopyEcShardsFromSource(ctx context.Context, grpcDialOption grpc.DialOption,
- targetServers []*EcNode, allocated []int,
+ targetServers []*EcNode, allocatedEcIds [][]uint32,
volumeId needle.VolumeId, collection string, existingLocation wdclient.Location) (actuallyCopied []uint32, err error) {
// parallelize
shardIdChan := make(chan []uint32, len(targetServers))
var wg sync.WaitGroup
- startFromShardId := uint32(0)
for i, server := range targetServers {
- if allocated[i] <= 0 {
+ if len(allocatedEcIds[i]) <= 0 {
continue
}
wg.Add(1)
- go func(server *EcNode, startFromShardId uint32, shardCount int) {
+ go func(server *EcNode, allocatedEcShardIds []uint32) {
defer wg.Done()
copiedShardIds, copyErr := oneServerCopyAndMountEcShardsFromSource(ctx, grpcDialOption, server,
- startFromShardId, shardCount, volumeId, collection, existingLocation.Url)
+ allocatedEcShardIds, volumeId, collection, existingLocation.Url)
if copyErr != nil {
err = copyErr
} else {
shardIdChan <- copiedShardIds
server.addEcVolumeShards(volumeId, collection, copiedShardIds)
}
- }(server, startFromShardId, allocated[i])
- startFromShardId += uint32(allocated[i])
+ }(server, allocatedEcIds[i])
}
wg.Wait()
close(shardIdChan)
@@ -236,18 +237,18 @@ func parallelCopyEcShardsFromSource(ctx context.Context, grpcDialOption grpc.Dia
return
}
-func balancedEcDistribution(servers []*EcNode) (allocated []int) {
- allocated = make([]int, len(servers))
- allocatedCount := 0
- for allocatedCount < erasure_coding.TotalShardsCount {
- for i, server := range servers {
- if server.freeEcSlot-allocated[i] > 0 {
- allocated[i] += 1
- allocatedCount += 1
- }
- if allocatedCount >= erasure_coding.TotalShardsCount {
- break
- }
+func balancedEcDistribution(servers []*EcNode) (allocated [][]uint32) {
+ allocated = make([][]uint32, len(servers))
+ allocatedShardIdIndex := uint32(0)
+ serverIndex := 0
+ for allocatedShardIdIndex < erasure_coding.TotalShardsCount {
+ if servers[serverIndex].freeEcSlot > 0 {
+ allocated[serverIndex] = append(allocated[serverIndex], allocatedShardIdIndex)
+ allocatedShardIdIndex++
+ }
+ serverIndex++
+ if serverIndex >= len(servers) {
+ serverIndex = 0
}
}
@@ -281,7 +282,7 @@ func collectVolumeIdsForEcEncode(ctx context.Context, commandEnv *CommandEnv, se
}
})
- for vid, _ := range vidMap {
+ for vid := range vidMap {
vids = append(vids, needle.VolumeId(vid))
}
diff --git a/weed/shell/command_ec_rebuild.go b/weed/shell/command_ec_rebuild.go
index 63b7c4088..2e2fca743 100644
--- a/weed/shell/command_ec_rebuild.go
+++ b/weed/shell/command_ec_rebuild.go
@@ -111,7 +111,7 @@ func rebuildEcVolumes(commandEnv *CommandEnv, allEcNodes []*EcNode, collection s
return fmt.Errorf("ec volume %d is unrepairable with %d shards\n", vid, shardCount)
}
- sortEcNodes(allEcNodes)
+ sortEcNodesByFreeslotsDecending(allEcNodes)
if allEcNodes[0].freeEcSlot < erasure_coding.TotalShardsCount {
return fmt.Errorf("disk space is not enough")
@@ -215,6 +215,8 @@ func prepareDataToRecover(ctx context.Context, commandEnv *CommandEnv, rebuilder
Collection: collection,
ShardIds: []uint32{uint32(shardId)},
CopyEcxFile: needEcxFile,
+ CopyEcjFile: needEcxFile,
+ CopyVifFile: needEcxFile,
SourceDataNode: ecNodes[0].info.Id,
})
return copyErr
diff --git a/weed/shell/command_ec_test.go b/weed/shell/command_ec_test.go
index 9e578ed28..c233d25d0 100644
--- a/weed/shell/command_ec_test.go
+++ b/weed/shell/command_ec_test.go
@@ -2,12 +2,25 @@ package shell
import (
"context"
+ "fmt"
"testing"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
)
+func TestCommandEcDistribution(t *testing.T) {
+
+ allEcNodes := []*EcNode{
+ newEcNode("dc1", "rack1", "dn1", 100),
+ newEcNode("dc1", "rack2", "dn2", 100),
+ }
+
+ allocated := balancedEcDistribution(allEcNodes)
+
+ fmt.Printf("allocated: %+v", allocated)
+}
+
func TestCommandEcBalanceSmall(t *testing.T) {
allEcNodes := []*EcNode{
diff --git a/weed/shell/command_fs_cat.go b/weed/shell/command_fs_cat.go
index 66ced46c5..9db36e9d1 100644
--- a/weed/shell/command_fs_cat.go
+++ b/weed/shell/command_fs_cat.go
@@ -24,12 +24,8 @@ func (c *commandFsCat) Name() string {
func (c *commandFsCat) Help() string {
return `stream the file content on to the screen
- fs.cat /dir/
fs.cat /dir/file_name
- fs.cat /dir/file_prefix
- fs.cat http://:/dir/
fs.cat http://:/dir/file_name
- fs.cat http://:/dir/file_prefix
`
}
diff --git a/weed/shell/command_fs_du.go b/weed/shell/command_fs_du.go
index 5e634c82a..1d7d79686 100644
--- a/weed/shell/command_fs_du.go
+++ b/weed/shell/command_fs_du.go
@@ -3,11 +3,13 @@ package shell
import (
"context"
"fmt"
+ "io"
+
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "google.golang.org/grpc"
- "io"
)
func init() {
@@ -43,67 +45,41 @@ func (c *commandFsDu) Do(args []string, commandEnv *CommandEnv, writer io.Writer
path = path + "/"
}
+ var blockCount, byteCount uint64
dir, name := filer2.FullPath(path).DirAndName()
+ blockCount, byteCount, err = duTraverseDirectory(ctx, writer, commandEnv.getFilerClient(filerServer, filerPort), dir, name)
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
-
- _, _, err = paginateDirectory(ctx, writer, client, dir, name, 1000)
-
- return err
+ if name == "" && err == nil {
+ fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s\n", blockCount, byteCount, dir)
+ }
- })
+ return
}
-func paginateDirectory(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, dir, name string, paginateSize int) (blockCount uint64, byteCount uint64, err error) {
-
- paginatedCount := -1
- startFromFileName := ""
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
- return
- }
+func duTraverseDirectory(ctx context.Context, writer io.Writer, filerClient filer2.FilerClient, dir, name string) (blockCount uint64, byteCount uint64, err error) {
- paginatedCount = len(resp.Entries)
-
- for _, entry := range resp.Entries {
- if entry.IsDirectory {
- subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
- if dir == "/" {
- subDir = "/" + entry.Name
- }
- numBlock, numByte, err := paginateDirectory(ctx, writer, client, subDir, "", paginateSize)
- if err == nil {
- blockCount += numBlock
- byteCount += numByte
- }
- } else {
- blockCount += uint64(len(entry.Chunks))
- byteCount += filer2.TotalSize(entry.Chunks)
+ err = filer2.ReadDirAllEntries(ctx, filerClient, dir, name, func(entry *filer_pb.Entry, isLast bool) {
+ if entry.IsDirectory {
+ subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
+ if dir == "/" {
+ subDir = "/" + entry.Name
}
- startFromFileName = entry.Name
-
- if name != "" && !entry.IsDirectory {
- fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s/%s\n", blockCount, byteCount, dir, name)
+ numBlock, numByte, err := duTraverseDirectory(ctx, writer, filerClient, subDir, "")
+ if err == nil {
+ blockCount += numBlock
+ byteCount += numByte
}
+ } else {
+ blockCount += uint64(len(entry.Chunks))
+ byteCount += filer2.TotalSize(entry.Chunks)
}
- }
-
- if name == "" {
- fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s\n", blockCount, byteCount, dir)
- }
+ if name != "" && !entry.IsDirectory {
+ fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s/%s\n", blockCount, byteCount, dir, name)
+ }
+ })
return
-
}
func (env *CommandEnv) withFilerClient(ctx context.Context, filerServer string, filerPort int64, fn func(filer_pb.SeaweedFilerClient) error) error {
@@ -115,3 +91,20 @@ func (env *CommandEnv) withFilerClient(ctx context.Context, filerServer string,
}, filerGrpcAddress, env.option.GrpcDialOption)
}
+
+type commandFilerClient struct {
+ env *CommandEnv
+ filerServer string
+ filerPort int64
+}
+
+func (env *CommandEnv) getFilerClient(filerServer string, filerPort int64) *commandFilerClient {
+ return &commandFilerClient{
+ env: env,
+ filerServer: filerServer,
+ filerPort: filerPort,
+ }
+}
+func (c *commandFilerClient) WithFilerClient(ctx context.Context, fn func(filer_pb.SeaweedFilerClient) error) error {
+ return c.env.withFilerClient(ctx, c.filerServer, c.filerPort, fn)
+}
diff --git a/weed/shell/command_fs_ls.go b/weed/shell/command_fs_ls.go
index 6979635e1..01842083b 100644
--- a/weed/shell/command_fs_ls.go
+++ b/weed/shell/command_fs_ls.go
@@ -3,13 +3,14 @@ package shell
import (
"context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"io"
"os"
"os/user"
"strconv"
"strings"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
func init() {
@@ -66,83 +67,51 @@ func (c *commandFsLs) Do(args []string, commandEnv *CommandEnv, writer io.Writer
}
dir, name := filer2.FullPath(path).DirAndName()
+ entryCount := 0
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
-
- return paginateOneDirectory(ctx, writer, client, dir, name, 1000, isLongFormat, showHidden)
-
- })
-
-}
-
-func paginateOneDirectory(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, dir, name string, paginateSize int, isLongFormat, showHidden bool) (err error) {
+ err = filer2.ReadDirAllEntries(ctx, commandEnv.getFilerClient(filerServer, filerPort), dir, name, func(entry *filer_pb.Entry, isLast bool) {
- entryCount := 0
- paginatedCount := -1
- startFromFileName := ""
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
+ if !showHidden && strings.HasPrefix(entry.Name, ".") {
return
}
- paginatedCount = len(resp.Entries)
-
- for _, entry := range resp.Entries {
+ entryCount++
- if !showHidden && strings.HasPrefix(entry.Name, ".") {
- continue
- }
-
- entryCount++
-
- if isLongFormat {
- fileMode := os.FileMode(entry.Attributes.FileMode)
- userName, groupNames := entry.Attributes.UserName, entry.Attributes.GroupName
- if userName == "" {
- if user, userErr := user.LookupId(strconv.Itoa(int(entry.Attributes.Uid))); userErr == nil {
- userName = user.Username
- }
- }
- groupName := ""
- if len(groupNames) > 0 {
- groupName = groupNames[0]
+ if isLongFormat {
+ fileMode := os.FileMode(entry.Attributes.FileMode)
+ userName, groupNames := entry.Attributes.UserName, entry.Attributes.GroupName
+ if userName == "" {
+ if user, userErr := user.LookupId(strconv.Itoa(int(entry.Attributes.Uid))); userErr == nil {
+ userName = user.Username
}
- if groupName == "" {
- if group, groupErr := user.LookupGroupId(strconv.Itoa(int(entry.Attributes.Gid))); groupErr == nil {
- groupName = group.Name
- }
- }
-
- if dir == "/" {
- // just for printing
- dir = ""
+ }
+ groupName := ""
+ if len(groupNames) > 0 {
+ groupName = groupNames[0]
+ }
+ if groupName == "" {
+ if group, groupErr := user.LookupGroupId(strconv.Itoa(int(entry.Attributes.Gid))); groupErr == nil {
+ groupName = group.Name
}
- fmt.Fprintf(writer, "%s %3d %s %s %6d %s/%s\n",
- fileMode, len(entry.Chunks),
- userName, groupName,
- filer2.TotalSize(entry.Chunks), dir, entry.Name)
- } else {
- fmt.Fprintf(writer, "%s\n", entry.Name)
}
- startFromFileName = entry.Name
-
+ if dir == "/" {
+ // just for printing
+ dir = ""
+ }
+ fmt.Fprintf(writer, "%s %3d %s %s %6d %s/%s\n",
+ fileMode, len(entry.Chunks),
+ userName, groupName,
+ filer2.TotalSize(entry.Chunks), dir, entry.Name)
+ } else {
+ fmt.Fprintf(writer, "%s\n", entry.Name)
}
- }
- if isLongFormat {
+ })
+
+ if isLongFormat && err == nil {
fmt.Fprintf(writer, "total %d\n", entryCount)
}
return
-
}
diff --git a/weed/shell/command_fs_meta_cat.go b/weed/shell/command_fs_meta_cat.go
new file mode 100644
index 000000000..5908b0a3c
--- /dev/null
+++ b/weed/shell/command_fs_meta_cat.go
@@ -0,0 +1,75 @@
+package shell
+
+import (
+ "context"
+ "fmt"
+ "io"
+
+ "github.com/golang/protobuf/jsonpb"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+)
+
+func init() {
+ Commands = append(Commands, &commandFsMetaCat{})
+}
+
+type commandFsMetaCat struct {
+}
+
+func (c *commandFsMetaCat) Name() string {
+ return "fs.meta.cat"
+}
+
+func (c *commandFsMetaCat) Help() string {
+ return `print out the meta data content for a file or directory
+
+ fs.meta.cat /dir/
+ fs.meta.cat /dir/file_name
+ fs.meta.cat http://:/dir/
+ fs.meta.cat http://:/dir/file_name
+`
+}
+
+func (c *commandFsMetaCat) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ input := findInputDirectory(args)
+
+ filerServer, filerPort, path, err := commandEnv.parseUrl(input)
+ if err != nil {
+ return err
+ }
+
+ ctx := context.Background()
+
+ dir, name := filer2.FullPath(path).DirAndName()
+
+ return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+
+ request := &filer_pb.LookupDirectoryEntryRequest{
+ Name: name,
+ Directory: dir,
+ }
+ respLookupEntry, err := client.LookupDirectoryEntry(ctx, request)
+ if err != nil {
+ return err
+ }
+
+ m := jsonpb.Marshaler{
+ EmitDefaults: true,
+ Indent: " ",
+ }
+
+ text, marshalErr := m.MarshalToString(respLookupEntry.Entry)
+ if marshalErr != nil {
+ return fmt.Errorf("marshal meta: %v", marshalErr)
+ }
+
+ fmt.Fprintf(writer, "%s\n", text)
+
+ return nil
+
+ })
+
+}
diff --git a/weed/shell/command_fs_meta_notify.go b/weed/shell/command_fs_meta_notify.go
index 13b272fbf..a898df7a0 100644
--- a/weed/shell/command_fs_meta_notify.go
+++ b/weed/shell/command_fs_meta_notify.go
@@ -5,11 +5,12 @@ import (
"fmt"
"io"
+ "github.com/spf13/viper"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/notification"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/spf13/viper"
)
func init() {
@@ -46,33 +47,33 @@ func (c *commandFsMetaNotify) Do(args []string, commandEnv *CommandEnv, writer i
ctx := context.Background()
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ var dirCount, fileCount uint64
- var dirCount, fileCount uint64
+ err = doTraverseBFS(ctx, writer, commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) {
- err = doTraverse(ctx, writer, client, filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) error {
-
- if entry.IsDirectory {
- dirCount++
- } else {
- fileCount++
- }
-
- return notification.Queue.SendMessage(
- string(parentPath.Child(entry.Name)),
- &filer_pb.EventNotification{
- NewEntry: entry,
- },
- )
+ if entry.IsDirectory {
+ dirCount++
+ } else {
+ fileCount++
+ }
- })
+ notifyErr := notification.Queue.SendMessage(
+ string(parentPath.Child(entry.Name)),
+ &filer_pb.EventNotification{
+ NewEntry: entry,
+ },
+ )
- if err == nil {
- fmt.Fprintf(writer, "\ntotal notified %d directories, %d files\n", dirCount, fileCount)
+ if notifyErr != nil {
+ fmt.Fprintf(writer, "fail to notify new entry event for %s: %v\n", parentPath.Child(entry.Name), notifyErr)
}
- return err
-
})
+ if err == nil {
+ fmt.Fprintf(writer, "\ntotal notified %d directories, %d files\n", dirCount, fileCount)
+ }
+
+ return err
+
}
diff --git a/weed/shell/command_fs_meta_save.go b/weed/shell/command_fs_meta_save.go
index 6ca395fae..ed070350f 100644
--- a/weed/shell/command_fs_meta_save.go
+++ b/weed/shell/command_fs_meta_save.go
@@ -2,15 +2,19 @@ package shell
import (
"context"
+ "flag"
"fmt"
"io"
"os"
+ "sync"
+ "sync/atomic"
"time"
+ "github.com/golang/protobuf/proto"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/golang/protobuf/proto"
)
func init() {
@@ -27,10 +31,11 @@ func (c *commandFsMetaSave) Name() string {
func (c *commandFsMetaSave) Help() string {
return `save all directory and file meta data to a local file for metadata backup.
- fs.meta.save / # save from the root
- fs.meta.save /path/to/save # save from the directory /path/to/save
- fs.meta.save . # save from current directory
- fs.meta.save # save from current directory
+ fs.meta.save / # save from the root
+ fs.meta.save -v -o t.meta / # save from the root, output to t.meta file.
+ fs.meta.save /path/to/save # save from the directory /path/to/save
+ fs.meta.save . # save from current directory
+ fs.meta.save # save from current directory
The meta data will be saved into a local --.meta file.
These meta data can be later loaded by fs.meta.load command,
@@ -42,109 +47,139 @@ func (c *commandFsMetaSave) Help() string {
func (c *commandFsMetaSave) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
- filerServer, filerPort, path, err := commandEnv.parseUrl(findInputDirectory(args))
- if err != nil {
- return err
+ fsMetaSaveCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ verbose := fsMetaSaveCommand.Bool("v", false, "print out each processed files")
+ outputFileName := fsMetaSaveCommand.String("o", "", "output the meta data to this file")
+ if err = fsMetaSaveCommand.Parse(args); err != nil {
+ return nil
}
- ctx := context.Background()
+ filerServer, filerPort, path, parseErr := commandEnv.parseUrl(findInputDirectory(fsMetaSaveCommand.Args()))
+ if parseErr != nil {
+ return parseErr
+ }
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ ctx := context.Background()
- t := time.Now()
- fileName := fmt.Sprintf("%s-%d-%4d%02d%02d-%02d%02d%02d.meta",
+ t := time.Now()
+ fileName := *outputFileName
+ if fileName == "" {
+ fileName = fmt.Sprintf("%s-%d-%4d%02d%02d-%02d%02d%02d.meta",
filerServer, filerPort, t.Year(), t.Month(), t.Day(), t.Hour(), t.Minute(), t.Second())
+ }
- dst, err := os.OpenFile(fileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
- if err != nil {
- return nil
- }
- defer dst.Close()
-
- var dirCount, fileCount uint64
+ dst, openErr := os.OpenFile(fileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if openErr != nil {
+ return fmt.Errorf("failed to create file %s: %v", fileName, openErr)
+ }
+ defer dst.Close()
+ var wg sync.WaitGroup
+ wg.Add(1)
+ outputChan := make(chan []byte, 1024)
+ go func() {
sizeBuf := make([]byte, 4)
+ for b := range outputChan {
+ util.Uint32toBytes(sizeBuf, uint32(len(b)))
+ dst.Write(sizeBuf)
+ dst.Write(b)
+ }
+ wg.Done()
+ }()
- err = doTraverse(ctx, writer, client, filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) error {
+ var dirCount, fileCount uint64
- protoMessage := &filer_pb.FullEntry{
- Dir: string(parentPath),
- Entry: entry,
- }
+ err = doTraverseBFS(ctx, writer, commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) {
- bytes, err := proto.Marshal(protoMessage)
- if err != nil {
- return fmt.Errorf("marshall error: %v", err)
- }
+ protoMessage := &filer_pb.FullEntry{
+ Dir: string(parentPath),
+ Entry: entry,
+ }
- util.Uint32toBytes(sizeBuf, uint32(len(bytes)))
+ bytes, err := proto.Marshal(protoMessage)
+ if err != nil {
+ fmt.Fprintf(writer, "marshall error: %v\n", err)
+ return
+ }
- dst.Write(sizeBuf)
- dst.Write(bytes)
+ outputChan <- bytes
- if entry.IsDirectory {
- dirCount++
- } else {
- fileCount++
- }
+ if entry.IsDirectory {
+ atomic.AddUint64(&dirCount, 1)
+ } else {
+ atomic.AddUint64(&fileCount, 1)
+ }
+ if *verbose {
println(parentPath.Child(entry.Name))
-
- return nil
-
- })
-
- if err == nil {
- fmt.Fprintf(writer, "\ntotal %d directories, %d files", dirCount, fileCount)
- fmt.Fprintf(writer, "\nmeta data for http://%s:%d%s is saved to %s\n", filerServer, filerPort, path, fileName)
}
- return err
-
})
-}
-func doTraverse(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, parentPath filer2.FullPath, fn func(parentPath filer2.FullPath, entry *filer_pb.Entry) error) (err error) {
-
- paginatedCount := -1
- startFromFileName := ""
- paginateSize := 1000
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: string(parentPath),
- Prefix: "",
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
- return
- }
+ close(outputChan)
- paginatedCount = len(resp.Entries)
+ wg.Wait()
- for _, entry := range resp.Entries {
+ if err == nil {
+ fmt.Fprintf(writer, "total %d directories, %d files\n", dirCount, fileCount)
+ fmt.Fprintf(writer, "meta data for http://%s:%d%s is saved to %s\n", filerServer, filerPort, path, fileName)
+ }
- if err = fn(parentPath, entry); err != nil {
- return err
- }
+ return err
- if entry.IsDirectory {
- subDir := fmt.Sprintf("%s/%s", parentPath, entry.Name)
- if parentPath == "/" {
- subDir = "/" + entry.Name
+}
+func doTraverseBFS(ctx context.Context, writer io.Writer, filerClient filer2.FilerClient,
+ parentPath filer2.FullPath, fn func(parentPath filer2.FullPath, entry *filer_pb.Entry)) (err error) {
+
+ K := 5
+
+ var jobQueueWg sync.WaitGroup
+ queue := util.NewQueue()
+ jobQueueWg.Add(1)
+ queue.Enqueue(parentPath)
+ var isTerminating bool
+
+ for i := 0; i < K; i++ {
+ go func() {
+ for {
+ if isTerminating {
+ break
+ }
+ t := queue.Dequeue()
+ if t == nil {
+ time.Sleep(329 * time.Millisecond)
+ continue
}
- if err = doTraverse(ctx, writer, client, filer2.FullPath(subDir), fn); err != nil {
- return err
+ dir := t.(filer2.FullPath)
+ processErr := processOneDirectory(ctx, writer, filerClient, dir, queue, &jobQueueWg, fn)
+ if processErr != nil {
+ err = processErr
}
+ jobQueueWg.Done()
}
- startFromFileName = entry.Name
-
- }
+ }()
}
-
+ jobQueueWg.Wait()
+ isTerminating = true
return
+}
+
+func processOneDirectory(ctx context.Context, writer io.Writer, filerClient filer2.FilerClient,
+ parentPath filer2.FullPath, queue *util.Queue, jobQueueWg *sync.WaitGroup,
+ fn func(parentPath filer2.FullPath, entry *filer_pb.Entry)) (err error) {
+
+ return filer2.ReadDirAllEntries(ctx, filerClient, string(parentPath), "", func(entry *filer_pb.Entry, isLast bool) {
+
+ fn(parentPath, entry)
+
+ if entry.IsDirectory {
+ subDir := fmt.Sprintf("%s/%s", parentPath, entry.Name)
+ if parentPath == "/" {
+ subDir = "/" + entry.Name
+ }
+ jobQueueWg.Add(1)
+ queue.Enqueue(filer2.FullPath(subDir))
+ }
+ })
}
diff --git a/weed/shell/command_fs_mv.go b/weed/shell/command_fs_mv.go
index 130bfe4e7..67606ab53 100644
--- a/weed/shell/command_fs_mv.go
+++ b/weed/shell/command_fs_mv.go
@@ -49,12 +49,10 @@ func (c *commandFsMv) Do(args []string, commandEnv *CommandEnv, writer io.Writer
ctx := context.Background()
-
sourceDir, sourceName := filer2.FullPath(sourcePath).DirAndName()
destinationDir, destinationName := filer2.FullPath(destinationPath).DirAndName()
-
return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
// collect destination entry info
@@ -77,7 +75,6 @@ func (c *commandFsMv) Do(args []string, commandEnv *CommandEnv, writer io.Writer
targetName = destinationName
}
-
request := &filer_pb.AtomicRenameEntryRequest{
OldDirectory: sourceDir,
OldName: sourceName,
diff --git a/weed/shell/command_fs_tree.go b/weed/shell/command_fs_tree.go
index 8474e43ea..a4524f341 100644
--- a/weed/shell/command_fs_tree.go
+++ b/weed/shell/command_fs_tree.go
@@ -3,10 +3,11 @@ package shell
import (
"context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"io"
"strings"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
func init() {
@@ -38,75 +39,45 @@ func (c *commandFsTree) Do(args []string, commandEnv *CommandEnv, writer io.Writ
ctx := context.Background()
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ dirCount, fCount, terr := treeTraverseDirectory(ctx, writer, commandEnv.getFilerClient(filerServer, filerPort), dir, name, newPrefix(), -1)
- dirCount, fCount, terr := treeTraverseDirectory(ctx, writer, client, dir, name, newPrefix(), -1)
-
- if terr == nil {
- fmt.Fprintf(writer, "%d directories, %d files\n", dirCount, fCount)
- }
-
- return terr
+ if terr == nil {
+ fmt.Fprintf(writer, "%d directories, %d files\n", dirCount, fCount)
+ }
- })
+ return terr
}
-func treeTraverseDirectory(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, dir, name string, prefix *Prefix, level int) (directoryCount, fileCount int64, err error) {
-
- paginatedCount := -1
- startFromFileName := ""
- paginateSize := 1000
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
- return
- }
- paginatedCount = len(resp.Entries)
- if paginatedCount > 0 {
- prefix.addMarker(level)
- }
+func treeTraverseDirectory(ctx context.Context, writer io.Writer, filerClient filer2.FilerClient, dir, name string, prefix *Prefix, level int) (directoryCount, fileCount int64, err error) {
- for i, entry := range resp.Entries {
+ prefix.addMarker(level)
- if level < 0 && name != "" {
- if entry.Name != name {
- break
- }
+ err = filer2.ReadDirAllEntries(ctx, filerClient, dir, name, func(entry *filer_pb.Entry, isLast bool) {
+ if level < 0 && name != "" {
+ if entry.Name != name {
+ return
}
+ }
- // 0.1% wrong prefix here, but fixing it would need to paginate to the next batch first
- isLast := paginatedCount < paginateSize && i == paginatedCount-1
- fmt.Fprintf(writer, "%s%s\n", prefix.getPrefix(level, isLast), entry.Name)
-
- if entry.IsDirectory {
- directoryCount++
- subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
- if dir == "/" {
- subDir = "/" + entry.Name
- }
- dirCount, fCount, terr := treeTraverseDirectory(ctx, writer, client, subDir, "", prefix, level+1)
- directoryCount += dirCount
- fileCount += fCount
- err = terr
- } else {
- fileCount++
- }
- startFromFileName = entry.Name
+ fmt.Fprintf(writer, "%s%s\n", prefix.getPrefix(level, isLast), entry.Name)
+ if entry.IsDirectory {
+ directoryCount++
+ subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
+ if dir == "/" {
+ subDir = "/" + entry.Name
+ }
+ dirCount, fCount, terr := treeTraverseDirectory(ctx, writer, filerClient, subDir, "", prefix, level+1)
+ directoryCount += dirCount
+ fileCount += fCount
+ err = terr
+ } else {
+ fileCount++
}
- }
+ })
return
-
}
type Prefix struct {
diff --git a/weed/shell/command_volume_balance.go b/weed/shell/command_volume_balance.go
index d7ef0d005..bed4f4306 100644
--- a/weed/shell/command_volume_balance.go
+++ b/weed/shell/command_volume_balance.go
@@ -27,7 +27,7 @@ func (c *commandVolumeBalance) Name() string {
func (c *commandVolumeBalance) Help() string {
return `balance all volumes among volume servers
- volume.balance [-c ALL|EACH_COLLECTION|] [-force] [-dataCenter=]
+ volume.balance [-collection ALL|EACH_COLLECTION|] [-force] [-dataCenter=]
Algorithm:
@@ -79,8 +79,10 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
}
typeToNodes := collectVolumeServersByType(resp.TopologyInfo, *dc)
- for _, volumeServers := range typeToNodes {
+
+ for maxVolumeCount, volumeServers := range typeToNodes {
if len(volumeServers) < 2 {
+ fmt.Printf("only 1 node is configured max %d volumes, skipping balancing\n", maxVolumeCount)
continue
}
if *collection == "EACH_COLLECTION" {
@@ -93,8 +95,8 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
return err
}
}
- } else if *collection == "ALL" {
- if err = balanceVolumeServers(commandEnv, volumeServers, resp.VolumeSizeLimitMb*1024*1024, "ALL", *applyBalancing); err != nil {
+ } else if *collection == "ALL_COLLECTIONS" {
+ if err = balanceVolumeServers(commandEnv, volumeServers, resp.VolumeSizeLimitMb*1024*1024, "ALL_COLLECTIONS", *applyBalancing); err != nil {
return err
}
} else {
@@ -108,6 +110,7 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
}
func balanceVolumeServers(commandEnv *CommandEnv, dataNodeInfos []*master_pb.DataNodeInfo, volumeSizeLimit uint64, collection string, applyBalancing bool) error {
+
var nodes []*Node
for _, dn := range dataNodeInfos {
nodes = append(nodes, &Node{
@@ -118,7 +121,7 @@ func balanceVolumeServers(commandEnv *CommandEnv, dataNodeInfos []*master_pb.Dat
// balance writable volumes
for _, n := range nodes {
n.selectVolumes(func(v *master_pb.VolumeInformationMessage) bool {
- if collection != "ALL" {
+ if collection != "ALL_COLLECTIONS" {
if v.Collection != collection {
return false
}
@@ -133,7 +136,7 @@ func balanceVolumeServers(commandEnv *CommandEnv, dataNodeInfos []*master_pb.Dat
// balance readable volumes
for _, n := range nodes {
n.selectVolumes(func(v *master_pb.VolumeInformationMessage) bool {
- if collection != "ALL" {
+ if collection != "ALL_COLLECTIONS" {
if v.Collection != collection {
return false
}
diff --git a/weed/shell/command_volume_fix_replication.go b/weed/shell/command_volume_fix_replication.go
index 4c7a794c0..6f35dd5d2 100644
--- a/weed/shell/command_volume_fix_replication.go
+++ b/weed/shell/command_volume_fix_replication.go
@@ -3,13 +3,14 @@ package shell
import (
"context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/operation"
- "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
- "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
- "github.com/chrislusf/seaweedfs/weed/storage"
"io"
"math/rand"
"sort"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
func init() {
@@ -78,7 +79,7 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
underReplicatedVolumeLocations := make(map[uint32][]location)
for vid, locations := range replicatedVolumeLocations {
volumeInfo := replicatedVolumeInfo[vid]
- replicaPlacement, _ := storage.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
+ replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
if replicaPlacement.GetCopyCount() > len(locations) {
underReplicatedVolumeLocations[vid] = locations
}
@@ -97,7 +98,7 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
for vid, locations := range underReplicatedVolumeLocations {
volumeInfo := replicatedVolumeInfo[vid]
- replicaPlacement, _ := storage.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
+ replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
foundNewLocation := false
for _, dst := range allLocations {
// check whether data nodes satisfy the constraints
@@ -145,7 +146,7 @@ func keepDataNodesSorted(dataNodes []location) {
})
}
-func satisfyReplicaPlacement(replicaPlacement *storage.ReplicaPlacement, existingLocations []location, possibleLocation location) bool {
+func satisfyReplicaPlacement(replicaPlacement *super_block.ReplicaPlacement, existingLocations []location, possibleLocation location) bool {
existingDataCenters := make(map[string]bool)
existingRacks := make(map[string]bool)
diff --git a/weed/shell/command_volume_list.go b/weed/shell/command_volume_list.go
index 134580ffe..c6c79d150 100644
--- a/weed/shell/command_volume_list.go
+++ b/weed/shell/command_volume_list.go
@@ -46,7 +46,7 @@ func (c *commandVolumeList) Do(args []string, commandEnv *CommandEnv, writer io.
}
func writeTopologyInfo(writer io.Writer, t *master_pb.TopologyInfo, volumeSizeLimitMb uint64) statistics {
- fmt.Fprintf(writer, "Topology volume:%d/%d active:%d free:%d volumeSizeLimit:%d MB\n", t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, volumeSizeLimitMb)
+ fmt.Fprintf(writer, "Topology volume:%d/%d active:%d free:%d remote:%d volumeSizeLimit:%d MB\n", t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount, volumeSizeLimitMb)
sort.Slice(t.DataCenterInfos, func(i, j int) bool {
return t.DataCenterInfos[i].Id < t.DataCenterInfos[j].Id
})
@@ -58,7 +58,7 @@ func writeTopologyInfo(writer io.Writer, t *master_pb.TopologyInfo, volumeSizeLi
return s
}
func writeDataCenterInfo(writer io.Writer, t *master_pb.DataCenterInfo) statistics {
- fmt.Fprintf(writer, " DataCenter %s volume:%d/%d active:%d free:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount)
+ fmt.Fprintf(writer, " DataCenter %s volume:%d/%d active:%d free:%d remote:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount)
var s statistics
sort.Slice(t.RackInfos, func(i, j int) bool {
return t.RackInfos[i].Id < t.RackInfos[j].Id
@@ -70,7 +70,7 @@ func writeDataCenterInfo(writer io.Writer, t *master_pb.DataCenterInfo) statisti
return s
}
func writeRackInfo(writer io.Writer, t *master_pb.RackInfo) statistics {
- fmt.Fprintf(writer, " Rack %s volume:%d/%d active:%d free:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount)
+ fmt.Fprintf(writer, " Rack %s volume:%d/%d active:%d free:%d remote:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount)
var s statistics
sort.Slice(t.DataNodeInfos, func(i, j int) bool {
return t.DataNodeInfos[i].Id < t.DataNodeInfos[j].Id
@@ -82,7 +82,7 @@ func writeRackInfo(writer io.Writer, t *master_pb.RackInfo) statistics {
return s
}
func writeDataNodeInfo(writer io.Writer, t *master_pb.DataNodeInfo) statistics {
- fmt.Fprintf(writer, " DataNode %s volume:%d/%d active:%d free:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount)
+ fmt.Fprintf(writer, " DataNode %s volume:%d/%d active:%d free:%d remote:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount)
var s statistics
sort.Slice(t.VolumeInfos, func(i, j int) bool {
return t.VolumeInfos[i].Id < t.VolumeInfos[j].Id
@@ -98,7 +98,7 @@ func writeDataNodeInfo(writer io.Writer, t *master_pb.DataNodeInfo) statistics {
}
func writeVolumeInformationMessage(writer io.Writer, t *master_pb.VolumeInformationMessage) statistics {
fmt.Fprintf(writer, " volume %+v \n", t)
- return newStatiscis(t)
+ return newStatistics(t)
}
type statistics struct {
@@ -108,7 +108,7 @@ type statistics struct {
DeletedBytes uint64
}
-func newStatiscis(t *master_pb.VolumeInformationMessage) statistics {
+func newStatistics(t *master_pb.VolumeInformationMessage) statistics {
return statistics{
Size: t.Size,
FileCount: t.FileCount,
diff --git a/weed/shell/command_volume_tier_download.go b/weed/shell/command_volume_tier_download.go
new file mode 100644
index 000000000..88e2e8b92
--- /dev/null
+++ b/weed/shell/command_volume_tier_download.go
@@ -0,0 +1,167 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+func init() {
+ Commands = append(Commands, &commandVolumeTierDownload{})
+}
+
+type commandVolumeTierDownload struct {
+}
+
+func (c *commandVolumeTierDownload) Name() string {
+ return "volume.tier.download"
+}
+
+func (c *commandVolumeTierDownload) Help() string {
+ return `move the dat file of a volume to a remote tier
+
+ volume.tier.download [-collection=""]
+ volume.tier.download [-collection=""] -volumeId=
+
+ e.g.:
+ volume.tier.download -volumeId=7
+ volume.tier.download -volumeId=7
+
+ This command will download the dat file of a volume from a remote tier to a volume server in local cluster.
+
+`
+}
+
+func (c *commandVolumeTierDownload) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ tierCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ volumeId := tierCommand.Int("volumeId", 0, "the volume id")
+ collection := tierCommand.String("collection", "", "the collection name")
+ if err = tierCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ ctx := context.Background()
+ vid := needle.VolumeId(*volumeId)
+
+ // collect topology information
+ topologyInfo, err := collectTopologyInfo(ctx, commandEnv)
+ if err != nil {
+ return err
+ }
+
+ // volumeId is provided
+ if vid != 0 {
+ return doVolumeTierDownload(ctx, commandEnv, writer, *collection, vid)
+ }
+
+ // apply to all volumes in the collection
+ // reusing collectVolumeIdsForEcEncode for now
+ volumeIds := collectRemoteVolumes(topologyInfo, *collection)
+ if err != nil {
+ return err
+ }
+ fmt.Printf("tier download volumes: %v\n", volumeIds)
+ for _, vid := range volumeIds {
+ if err = doVolumeTierDownload(ctx, commandEnv, writer, *collection, vid); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
+func collectRemoteVolumes(topoInfo *master_pb.TopologyInfo, selectedCollection string) (vids []needle.VolumeId) {
+
+ vidMap := make(map[uint32]bool)
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.VolumeInfos {
+ if v.Collection == selectedCollection && v.RemoteStorageKey != "" && v.RemoteStorageName != "" {
+ vidMap[v.Id] = true
+ }
+ }
+ })
+
+ for vid := range vidMap {
+ vids = append(vids, needle.VolumeId(vid))
+ }
+
+ return
+}
+
+func doVolumeTierDownload(ctx context.Context, commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId) (err error) {
+ // find volume location
+ locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
+ if !found {
+ return fmt.Errorf("volume %d not found", vid)
+ }
+
+ // TODO parallelize this
+ for _, loc := range locations {
+ // copy the .dat file from remote tier to local
+ err = downloadDatFromRemoteTier(ctx, commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, loc.Url)
+ if err != nil {
+ return fmt.Errorf("download dat file for volume %d to %s: %v", vid, loc.Url, err)
+ }
+ }
+
+ return nil
+}
+
+func downloadDatFromRemoteTier(ctx context.Context, grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, targetVolumeServer string) error {
+
+ err := operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ stream, downloadErr := volumeServerClient.VolumeTierMoveDatFromRemote(ctx, &volume_server_pb.VolumeTierMoveDatFromRemoteRequest{
+ VolumeId: uint32(volumeId),
+ Collection: collection,
+ })
+
+ var lastProcessed int64
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ break
+ } else {
+ return recvErr
+ }
+ }
+
+ processingSpeed := float64(resp.Processed-lastProcessed) / 1024.0 / 1024.0
+
+ fmt.Fprintf(writer, "downloaded %.2f%%, %d bytes, %.2fMB/s\n", resp.ProcessedPercentage, resp.Processed, processingSpeed)
+
+ lastProcessed = resp.Processed
+ }
+ if downloadErr != nil {
+ return downloadErr
+ }
+
+ _, unmountErr := volumeServerClient.VolumeUnmount(ctx, &volume_server_pb.VolumeUnmountRequest{
+ VolumeId: uint32(volumeId),
+ })
+ if unmountErr != nil {
+ return unmountErr
+ }
+
+ _, mountErr := volumeServerClient.VolumeMount(ctx, &volume_server_pb.VolumeMountRequest{
+ VolumeId: uint32(volumeId),
+ })
+ if mountErr != nil {
+ return mountErr
+ }
+
+ return nil
+ })
+
+ return err
+
+}
diff --git a/weed/shell/command_volume_tier_upload.go b/weed/shell/command_volume_tier_upload.go
new file mode 100644
index 000000000..b3a0d9fe8
--- /dev/null
+++ b/weed/shell/command_volume_tier_upload.go
@@ -0,0 +1,148 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+ "time"
+
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+func init() {
+ Commands = append(Commands, &commandVolumeTierUpload{})
+}
+
+type commandVolumeTierUpload struct {
+}
+
+func (c *commandVolumeTierUpload) Name() string {
+ return "volume.tier.upload"
+}
+
+func (c *commandVolumeTierUpload) Help() string {
+ return `move the dat file of a volume to a remote tier
+
+ volume.tier.upload [-collection=""] [-fullPercent=95] [-quietFor=1h]
+ volume.tier.upload [-collection=""] -volumeId= -dest= [-keepLocalDatFile]
+
+ e.g.:
+ volume.tier.upload -volumeId=7 -dest=s3
+ volume.tier.upload -volumeId=7 -dest=s3.default
+
+ The is defined in master.toml.
+ For example, "s3.default" in [storage.backend.s3.default]
+
+ This command will move the dat file of a volume to a remote tier.
+
+ SeaweedFS enables scalable and fast local access to lots of files,
+ and the cloud storage is slower by cost efficient. How to combine them together?
+
+ Usually the data follows 80/20 rule: only 20% of data is frequently accessed.
+ We can offload the old volumes to the cloud.
+
+ With this, SeaweedFS can be both fast and scalable, and infinite storage space.
+ Just add more local SeaweedFS volume servers to increase the throughput.
+
+ The index file is still local, and the same O(1) disk read is applied to the remote file.
+
+`
+}
+
+func (c *commandVolumeTierUpload) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ tierCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ volumeId := tierCommand.Int("volumeId", 0, "the volume id")
+ collection := tierCommand.String("collection", "", "the collection name")
+ fullPercentage := tierCommand.Float64("fullPercent", 95, "the volume reaches the percentage of max volume size")
+ quietPeriod := tierCommand.Duration("quietFor", 24*time.Hour, "select volumes without no writes for this period")
+ dest := tierCommand.String("dest", "", "the target tier name")
+ keepLocalDatFile := tierCommand.Bool("keepLocalDatFile", false, "whether keep local dat file")
+ if err = tierCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ ctx := context.Background()
+ vid := needle.VolumeId(*volumeId)
+
+ // volumeId is provided
+ if vid != 0 {
+ return doVolumeTierUpload(ctx, commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile)
+ }
+
+ // apply to all volumes in the collection
+ // reusing collectVolumeIdsForEcEncode for now
+ volumeIds, err := collectVolumeIdsForEcEncode(ctx, commandEnv, *collection, *fullPercentage, *quietPeriod)
+ if err != nil {
+ return err
+ }
+ fmt.Printf("tier upload volumes: %v\n", volumeIds)
+ for _, vid := range volumeIds {
+ if err = doVolumeTierUpload(ctx, commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
+func doVolumeTierUpload(ctx context.Context, commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId, dest string, keepLocalDatFile bool) (err error) {
+ // find volume location
+ locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
+ if !found {
+ return fmt.Errorf("volume %d not found", vid)
+ }
+
+ err = markVolumeReadonly(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
+ if err != nil {
+ return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
+ }
+
+ // copy the .dat file to remote tier
+ err = uploadDatToRemoteTier(ctx, commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, locations[0].Url, dest, keepLocalDatFile)
+ if err != nil {
+ return fmt.Errorf("copy dat file for volume %d on %s to %s: %v", vid, locations[0].Url, dest, err)
+ }
+
+ return nil
+}
+
+func uploadDatToRemoteTier(ctx context.Context, grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, sourceVolumeServer string, dest string, keepLocalDatFile bool) error {
+
+ err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ stream, copyErr := volumeServerClient.VolumeTierMoveDatToRemote(ctx, &volume_server_pb.VolumeTierMoveDatToRemoteRequest{
+ VolumeId: uint32(volumeId),
+ Collection: collection,
+ DestinationBackendName: dest,
+ KeepLocalDatFile: keepLocalDatFile,
+ })
+
+ var lastProcessed int64
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ break
+ } else {
+ return recvErr
+ }
+ }
+
+ processingSpeed := float64(resp.Processed-lastProcessed) / 1024.0 / 1024.0
+
+ fmt.Fprintf(writer, "copied %.2f%%, %d bytes, %.2fMB/s\n", resp.ProcessedPercentage, resp.Processed, processingSpeed)
+
+ lastProcessed = resp.Processed
+ }
+
+ return copyErr
+ })
+
+ return err
+
+}
diff --git a/weed/shell/commands.go b/weed/shell/commands.go
index b642ec253..a6a0f7dec 100644
--- a/weed/shell/commands.go
+++ b/weed/shell/commands.go
@@ -9,10 +9,11 @@ import (
"strconv"
"strings"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/wdclient"
- "google.golang.org/grpc"
)
type ShellOptions struct {
@@ -71,26 +72,19 @@ func (ce *CommandEnv) checkDirectory(ctx context.Context, filerServer string, fi
return ce.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: name,
- InclusiveStartFrom: true,
- Limit: 1,
+ resp, lookupErr := client.LookupDirectoryEntry(ctx, &filer_pb.LookupDirectoryEntryRequest{
+ Directory: dir,
+ Name: name,
})
- if listErr != nil {
- return listErr
+ if lookupErr != nil {
+ return lookupErr
}
- if len(resp.Entries) == 0 {
+ if resp.Entry == nil {
return fmt.Errorf("entry not found")
}
- if resp.Entries[0].Name != name {
- return fmt.Errorf("not a valid directory, found %s", resp.Entries[0].Name)
- }
-
- if !resp.Entries[0].IsDirectory {
+ if !resp.Entry.IsDirectory {
return fmt.Errorf("not a directory")
}
diff --git a/weed/storage/backend/backend.go b/weed/storage/backend/backend.go
new file mode 100644
index 000000000..6ea850543
--- /dev/null
+++ b/weed/storage/backend/backend.go
@@ -0,0 +1,137 @@
+package backend
+
+import (
+ "io"
+ "os"
+ "strings"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/spf13/viper"
+)
+
+type BackendStorageFile interface {
+ io.ReaderAt
+ io.WriterAt
+ Truncate(off int64) error
+ io.Closer
+ GetStat() (datSize int64, modTime time.Time, err error)
+ Name() string
+}
+
+type BackendStorage interface {
+ ToProperties() map[string]string
+ NewStorageFile(key string, tierInfo *volume_server_pb.VolumeInfo) BackendStorageFile
+ CopyFile(f *os.File, attributes map[string]string, fn func(progressed int64, percentage float32) error) (key string, size int64, err error)
+ DownloadFile(fileName string, key string, fn func(progressed int64, percentage float32) error) (size int64, err error)
+ DeleteFile(key string) (err error)
+}
+
+type StringProperties interface {
+ GetString(key string) string
+}
+type StorageType string
+type BackendStorageFactory interface {
+ StorageType() StorageType
+ BuildStorage(configuration StringProperties, id string) (BackendStorage, error)
+}
+
+var (
+ BackendStorageFactories = make(map[StorageType]BackendStorageFactory)
+ BackendStorages = make(map[string]BackendStorage)
+)
+
+// used by master to load remote storage configurations
+func LoadConfiguration(config *viper.Viper) {
+
+ StorageBackendPrefix := "storage.backend"
+
+ backendSub := config.Sub(StorageBackendPrefix)
+
+ for backendTypeName := range config.GetStringMap(StorageBackendPrefix) {
+ backendStorageFactory, found := BackendStorageFactories[StorageType(backendTypeName)]
+ if !found {
+ glog.Fatalf("backend storage type %s not found", backendTypeName)
+ }
+ backendTypeSub := backendSub.Sub(backendTypeName)
+ for backendStorageId := range backendSub.GetStringMap(backendTypeName) {
+ if !backendTypeSub.GetBool(backendStorageId + ".enabled") {
+ continue
+ }
+ backendStorage, buildErr := backendStorageFactory.BuildStorage(backendTypeSub.Sub(backendStorageId), backendStorageId)
+ if buildErr != nil {
+ glog.Fatalf("fail to create backend storage %s.%s", backendTypeName, backendStorageId)
+ }
+ BackendStorages[backendTypeName+"."+backendStorageId] = backendStorage
+ if backendStorageId == "default" {
+ BackendStorages[backendTypeName] = backendStorage
+ }
+ }
+ }
+
+}
+
+// used by volume server to receive remote storage configurations from master
+func LoadFromPbStorageBackends(storageBackends []*master_pb.StorageBackend) {
+
+ for _, storageBackend := range storageBackends {
+ backendStorageFactory, found := BackendStorageFactories[StorageType(storageBackend.Type)]
+ if !found {
+ glog.Warningf("storage type %s not found", storageBackend.Type)
+ continue
+ }
+ backendStorage, buildErr := backendStorageFactory.BuildStorage(newProperties(storageBackend.Properties), storageBackend.Id)
+ if buildErr != nil {
+ glog.Fatalf("fail to create backend storage %s.%s", storageBackend.Type, storageBackend.Id)
+ }
+ BackendStorages[storageBackend.Type+"."+storageBackend.Id] = backendStorage
+ if storageBackend.Id == "default" {
+ BackendStorages[storageBackend.Type] = backendStorage
+ }
+ }
+}
+
+type Properties struct {
+ m map[string]string
+}
+
+func newProperties(m map[string]string) *Properties {
+ return &Properties{m: m}
+}
+
+func (p *Properties) GetString(key string) string {
+ if v, found := p.m[key]; found {
+ return v
+ }
+ return ""
+}
+
+func ToPbStorageBackends() (backends []*master_pb.StorageBackend) {
+ for sName, s := range BackendStorages {
+ sType, sId := BackendNameToTypeId(sName)
+ if sType == "" {
+ continue
+ }
+ backends = append(backends, &master_pb.StorageBackend{
+ Type: sType,
+ Id: sId,
+ Properties: s.ToProperties(),
+ })
+ }
+ return
+}
+
+func BackendNameToTypeId(backendName string) (backendType, backendId string) {
+ parts := strings.Split(backendName, ".")
+ if len(parts) == 1 {
+ return backendName, "default"
+ }
+ if len(parts) != 2 {
+ return
+ }
+
+ backendType, backendId = parts[0], parts[1]
+ return
+}
diff --git a/weed/storage/backend/disk_file.go b/weed/storage/backend/disk_file.go
new file mode 100644
index 000000000..c4b3caffb
--- /dev/null
+++ b/weed/storage/backend/disk_file.go
@@ -0,0 +1,50 @@
+package backend
+
+import (
+ "os"
+ "time"
+)
+
+var (
+ _ BackendStorageFile = &DiskFile{}
+)
+
+type DiskFile struct {
+ File *os.File
+ fullFilePath string
+}
+
+func NewDiskFile(f *os.File) *DiskFile {
+ return &DiskFile{
+ fullFilePath: f.Name(),
+ File: f,
+ }
+}
+
+func (df *DiskFile) ReadAt(p []byte, off int64) (n int, err error) {
+ return df.File.ReadAt(p, off)
+}
+
+func (df *DiskFile) WriteAt(p []byte, off int64) (n int, err error) {
+ return df.File.WriteAt(p, off)
+}
+
+func (df *DiskFile) Truncate(off int64) error {
+ return df.File.Truncate(off)
+}
+
+func (df *DiskFile) Close() error {
+ return df.File.Close()
+}
+
+func (df *DiskFile) GetStat() (datSize int64, modTime time.Time, err error) {
+ stat, e := df.File.Stat()
+ if e == nil {
+ return stat.Size(), stat.ModTime(), nil
+ }
+ return 0, time.Time{}, err
+}
+
+func (df *DiskFile) Name() string {
+ return df.fullFilePath
+}
diff --git a/weed/storage/backend/memory_map/memory_map.go b/weed/storage/backend/memory_map/memory_map.go
new file mode 100644
index 000000000..5dc7ba33d
--- /dev/null
+++ b/weed/storage/backend/memory_map/memory_map.go
@@ -0,0 +1,30 @@
+package memory_map
+
+import (
+ "os"
+ "strconv"
+)
+
+type MemoryBuffer struct {
+ aligned_length uint64
+ length uint64
+ aligned_ptr uintptr
+ ptr uintptr
+ Buffer []byte
+}
+
+type MemoryMap struct {
+ File *os.File
+ file_memory_map_handle uintptr
+ write_map_views []MemoryBuffer
+ max_length uint64
+ End_of_file int64
+}
+
+func ReadMemoryMapMaxSizeMb(memoryMapMaxSizeMbString string) (uint32, error) {
+ if memoryMapMaxSizeMbString == "" {
+ return 0, nil
+ }
+ memoryMapMaxSize64, err := strconv.ParseUint(memoryMapMaxSizeMbString, 10, 32)
+ return uint32(memoryMapMaxSize64), err
+}
diff --git a/weed/storage/backend/memory_map/memory_map_backend.go b/weed/storage/backend/memory_map/memory_map_backend.go
new file mode 100644
index 000000000..03e7308d0
--- /dev/null
+++ b/weed/storage/backend/memory_map/memory_map_backend.go
@@ -0,0 +1,60 @@
+package memory_map
+
+import (
+ "os"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+)
+
+var (
+ _ backend.BackendStorageFile = &MemoryMappedFile{}
+)
+
+type MemoryMappedFile struct {
+ mm *MemoryMap
+}
+
+func NewMemoryMappedFile(f *os.File, memoryMapSizeMB uint32) *MemoryMappedFile {
+ mmf := &MemoryMappedFile{
+ mm: new(MemoryMap),
+ }
+ mmf.mm.CreateMemoryMap(f, 1024*1024*uint64(memoryMapSizeMB))
+ return mmf
+}
+
+func (mmf *MemoryMappedFile) ReadAt(p []byte, off int64) (n int, err error) {
+ readBytes, e := mmf.mm.ReadMemory(uint64(off), uint64(len(p)))
+ if e != nil {
+ return 0, e
+ }
+ // TODO avoid the extra copy
+ copy(p, readBytes)
+ return len(readBytes), nil
+}
+
+func (mmf *MemoryMappedFile) WriteAt(p []byte, off int64) (n int, err error) {
+ mmf.mm.WriteMemory(uint64(off), uint64(len(p)), p)
+ return len(p), nil
+}
+
+func (mmf *MemoryMappedFile) Truncate(off int64) error {
+ return nil
+}
+
+func (mmf *MemoryMappedFile) Close() error {
+ mmf.mm.DeleteFileAndMemoryMap()
+ return nil
+}
+
+func (mmf *MemoryMappedFile) GetStat() (datSize int64, modTime time.Time, err error) {
+ stat, e := mmf.mm.File.Stat()
+ if e == nil {
+ return mmf.mm.End_of_file + 1, stat.ModTime(), nil
+ }
+ return 0, time.Time{}, err
+}
+
+func (mmf *MemoryMappedFile) Name() string {
+ return mmf.mm.File.Name()
+}
diff --git a/weed/storage/backend/memory_map/memory_map_other.go b/weed/storage/backend/memory_map/memory_map_other.go
new file mode 100644
index 000000000..e06a0b59a
--- /dev/null
+++ b/weed/storage/backend/memory_map/memory_map_other.go
@@ -0,0 +1,24 @@
+// +build !windows
+
+package memory_map
+
+import (
+ "fmt"
+ "os"
+)
+
+func (mMap *MemoryMap) CreateMemoryMap(file *os.File, maxLength uint64) {
+}
+
+func (mMap *MemoryMap) WriteMemory(offset uint64, length uint64, data []byte) {
+
+}
+
+func (mMap *MemoryMap) ReadMemory(offset uint64, length uint64) ([]byte, error) {
+ dataSlice := []byte{}
+ return dataSlice, fmt.Errorf("Memory Map not implemented for this platform")
+}
+
+func (mBuffer *MemoryMap) DeleteFileAndMemoryMap() {
+
+}
diff --git a/weed/storage/backend/memory_map/memory_map_test.go b/weed/storage/backend/memory_map/memory_map_test.go
new file mode 100644
index 000000000..33e1a828c
--- /dev/null
+++ b/weed/storage/backend/memory_map/memory_map_test.go
@@ -0,0 +1,10 @@
+package memory_map
+
+import "testing"
+
+func TestMemoryMapMaxSizeReadWrite(t *testing.T) {
+ memoryMapSize, _ := ReadMemoryMapMaxSizeMb("5000")
+ if memoryMapSize != 5000 {
+ t.Errorf("empty memoryMapSize:%v", memoryMapSize)
+ }
+}
diff --git a/weed/storage/backend/memory_map/memory_map_windows.go b/weed/storage/backend/memory_map/memory_map_windows.go
new file mode 100644
index 000000000..7eb713442
--- /dev/null
+++ b/weed/storage/backend/memory_map/memory_map_windows.go
@@ -0,0 +1,325 @@
+// +build windows
+
+package memory_map
+
+import (
+ "os"
+ "reflect"
+ "syscall"
+ "unsafe"
+
+ "golang.org/x/sys/windows"
+)
+
+type DWORDLONG = uint64
+type DWORD = uint32
+type WORD = uint16
+
+var (
+ modkernel32 = syscall.NewLazyDLL("kernel32.dll")
+
+ procGetSystemInfo = modkernel32.NewProc("GetSystemInfo")
+ procGlobalMemoryStatusEx = modkernel32.NewProc("GlobalMemoryStatusEx")
+ procGetProcessWorkingSetSize = modkernel32.NewProc("GetProcessWorkingSetSize")
+ procSetProcessWorkingSetSize = modkernel32.NewProc("SetProcessWorkingSetSize")
+)
+
+var currentProcess, _ = windows.GetCurrentProcess()
+var currentMinWorkingSet uint64 = 0
+var currentMaxWorkingSet uint64 = 0
+var _ = getProcessWorkingSetSize(uintptr(currentProcess), ¤tMinWorkingSet, ¤tMaxWorkingSet)
+
+var systemInfo, _ = getSystemInfo()
+var chunkSize = uint64(systemInfo.dwAllocationGranularity) * 128
+
+var memoryStatusEx, _ = globalMemoryStatusEx()
+var maxMemoryLimitBytes = uint64(float64(memoryStatusEx.ullTotalPhys) * 0.8)
+
+func (mMap *MemoryMap) CreateMemoryMap(file *os.File, maxLength uint64) {
+
+ chunks := (maxLength / chunkSize)
+ if chunks*chunkSize < maxLength {
+ chunks = chunks + 1
+ }
+
+ alignedMaxLength := chunks * chunkSize
+
+ maxLength_high := uint32(alignedMaxLength >> 32)
+ maxLength_low := uint32(alignedMaxLength & 0xFFFFFFFF)
+ file_memory_map_handle, err := windows.CreateFileMapping(windows.Handle(file.Fd()), nil, windows.PAGE_READWRITE, maxLength_high, maxLength_low, nil)
+
+ if err == nil {
+ mMap.File = file
+ mMap.file_memory_map_handle = uintptr(file_memory_map_handle)
+ mMap.write_map_views = make([]MemoryBuffer, 0, alignedMaxLength/chunkSize)
+ mMap.max_length = alignedMaxLength
+ mMap.End_of_file = -1
+ }
+}
+
+func (mMap *MemoryMap) DeleteFileAndMemoryMap() {
+ //First we close the file handles first to delete the file,
+ //Then we unmap the memory to ensure the unmapping process doesn't write the data to disk
+ windows.CloseHandle(windows.Handle(mMap.file_memory_map_handle))
+ windows.CloseHandle(windows.Handle(mMap.File.Fd()))
+
+ for _, view := range mMap.write_map_views {
+ view.releaseMemory()
+ }
+
+ mMap.write_map_views = nil
+ mMap.max_length = 0
+}
+
+func min(x, y uint64) uint64 {
+ if x < y {
+ return x
+ }
+ return y
+}
+
+func (mMap *MemoryMap) WriteMemory(offset uint64, length uint64, data []byte) {
+
+ for {
+ if ((offset+length)/chunkSize)+1 > uint64(len(mMap.write_map_views)) {
+ allocateChunk(mMap)
+ } else {
+ break
+ }
+ }
+
+ remaining_length := length
+ sliceIndex := offset / chunkSize
+ sliceOffset := offset - (sliceIndex * chunkSize)
+ dataOffset := uint64(0)
+
+ for {
+ writeEnd := min((remaining_length + sliceOffset), chunkSize)
+ copy(mMap.write_map_views[sliceIndex].Buffer[sliceOffset:writeEnd], data[dataOffset:])
+ remaining_length -= (writeEnd - sliceOffset)
+ dataOffset += (writeEnd - sliceOffset)
+
+ if remaining_length > 0 {
+ sliceIndex += 1
+ sliceOffset = 0
+ } else {
+ break
+ }
+ }
+
+ if mMap.End_of_file < int64(offset+length-1) {
+ mMap.End_of_file = int64(offset + length - 1)
+ }
+}
+
+func (mMap *MemoryMap) ReadMemory(offset uint64, length uint64) (dataSlice []byte, err error) {
+ dataSlice = make([]byte, length)
+ mBuffer, err := allocate(windows.Handle(mMap.file_memory_map_handle), offset, length, false)
+ copy(dataSlice, mBuffer.Buffer)
+ mBuffer.releaseMemory()
+ return dataSlice, err
+}
+
+func (mBuffer *MemoryBuffer) releaseMemory() {
+
+ windows.VirtualUnlock(mBuffer.aligned_ptr, uintptr(mBuffer.aligned_length))
+ windows.UnmapViewOfFile(mBuffer.aligned_ptr)
+
+ currentMinWorkingSet -= mBuffer.aligned_length
+ currentMaxWorkingSet -= mBuffer.aligned_length
+
+ if currentMinWorkingSet < maxMemoryLimitBytes {
+ var _ = setProcessWorkingSetSize(uintptr(currentProcess), currentMinWorkingSet, currentMaxWorkingSet)
+ }
+
+ mBuffer.ptr = 0
+ mBuffer.aligned_ptr = 0
+ mBuffer.length = 0
+ mBuffer.aligned_length = 0
+ mBuffer.Buffer = nil
+}
+
+func allocateChunk(mMap *MemoryMap) {
+ start := uint64(len(mMap.write_map_views)) * chunkSize
+ mBuffer, err := allocate(windows.Handle(mMap.file_memory_map_handle), start, chunkSize, true)
+
+ if err == nil {
+ mMap.write_map_views = append(mMap.write_map_views, mBuffer)
+ }
+}
+
+func allocate(hMapFile windows.Handle, offset uint64, length uint64, write bool) (MemoryBuffer, error) {
+
+ mBuffer := MemoryBuffer{}
+
+ //align memory allocations to the minium virtal memory allocation size
+ dwSysGran := systemInfo.dwAllocationGranularity
+
+ start := (offset / uint64(dwSysGran)) * uint64(dwSysGran)
+ diff := offset - start
+ aligned_length := diff + length
+
+ offset_high := uint32(start >> 32)
+ offset_low := uint32(start & 0xFFFFFFFF)
+
+ access := windows.FILE_MAP_READ
+
+ if write {
+ access = windows.FILE_MAP_WRITE
+ }
+
+ currentMinWorkingSet += aligned_length
+ currentMaxWorkingSet += aligned_length
+
+ if currentMinWorkingSet < maxMemoryLimitBytes {
+ // increase the process working set size to hint to windows memory manager to
+ // prioritise keeping this memory mapped in physical memory over other standby memory
+ var _ = setProcessWorkingSetSize(uintptr(currentProcess), currentMinWorkingSet, currentMaxWorkingSet)
+ }
+
+ addr_ptr, errno := windows.MapViewOfFile(hMapFile,
+ uint32(access), // read/write permission
+ offset_high,
+ offset_low,
+ uintptr(aligned_length))
+
+ if addr_ptr == 0 {
+ return mBuffer, errno
+ }
+
+ if currentMinWorkingSet < maxMemoryLimitBytes {
+ windows.VirtualLock(mBuffer.aligned_ptr, uintptr(mBuffer.aligned_length))
+ }
+
+ mBuffer.aligned_ptr = addr_ptr
+ mBuffer.aligned_length = aligned_length
+ mBuffer.ptr = addr_ptr + uintptr(diff)
+ mBuffer.length = length
+
+ slice_header := (*reflect.SliceHeader)(unsafe.Pointer(&mBuffer.Buffer))
+ slice_header.Data = addr_ptr + uintptr(diff)
+ slice_header.Len = int(length)
+ slice_header.Cap = int(length)
+
+ return mBuffer, nil
+}
+
+//typedef struct _MEMORYSTATUSEX {
+// DWORD dwLength;
+// DWORD dwMemoryLoad;
+// DWORDLONG ullTotalPhys;
+// DWORDLONG ullAvailPhys;
+// DWORDLONG ullTotalPageFile;
+// DWORDLONG ullAvailPageFile;
+// DWORDLONG ullTotalVirtual;
+// DWORDLONG ullAvailVirtual;
+// DWORDLONG ullAvailExtendedVirtual;
+// } MEMORYSTATUSEX, *LPMEMORYSTATUSEX;
+//https://docs.microsoft.com/en-gb/windows/win32/api/sysinfoapi/ns-sysinfoapi-memorystatusex
+
+type _MEMORYSTATUSEX struct {
+ dwLength DWORD
+ dwMemoryLoad DWORD
+ ullTotalPhys DWORDLONG
+ ullAvailPhys DWORDLONG
+ ullTotalPageFile DWORDLONG
+ ullAvailPageFile DWORDLONG
+ ullTotalVirtual DWORDLONG
+ ullAvailVirtual DWORDLONG
+ ullAvailExtendedVirtual DWORDLONG
+}
+
+// BOOL GlobalMemoryStatusEx(
+// LPMEMORYSTATUSEX lpBuffer
+// );
+// https://docs.microsoft.com/en-gb/windows/win32/api/sysinfoapi/nf-sysinfoapi-globalmemorystatusex
+func globalMemoryStatusEx() (_MEMORYSTATUSEX, error) {
+ var mem_status _MEMORYSTATUSEX
+
+ mem_status.dwLength = uint32(unsafe.Sizeof(mem_status))
+ _, _, err := procGlobalMemoryStatusEx.Call(uintptr(unsafe.Pointer(&mem_status)))
+
+ if err != syscall.Errno(0) {
+ return mem_status, err
+ }
+ return mem_status, nil
+}
+
+// typedef struct _SYSTEM_INFO {
+// union {
+// DWORD dwOemId;
+// struct {
+// WORD wProcessorArchitecture;
+// WORD wReserved;
+// };
+// };
+// DWORD dwPageSize;
+// LPVOID lpMinimumApplicationAddress;
+// LPVOID lpMaximumApplicationAddress;
+// DWORD_PTR dwActiveProcessorMask;
+// DWORD dwNumberOfProcessors;
+// DWORD dwProcessorType;
+// DWORD dwAllocationGranularity;
+// WORD wProcessorLevel;
+// WORD wProcessorRevision;
+// } SYSTEM_INFO;
+// https://docs.microsoft.com/en-gb/windows/win32/api/sysinfoapi/ns-sysinfoapi-system_info
+type _SYSTEM_INFO struct {
+ dwOemId DWORD
+ dwPageSize DWORD
+ lpMinimumApplicationAddress uintptr
+ lpMaximumApplicationAddress uintptr
+ dwActiveProcessorMask uintptr
+ dwNumberOfProcessors DWORD
+ dwProcessorType DWORD
+ dwAllocationGranularity DWORD
+ wProcessorLevel WORD
+ wProcessorRevision WORD
+}
+
+// void WINAPI GetSystemInfo(
+// _Out_ LPSYSTEM_INFO lpSystemInfo
+// );
+// https://docs.microsoft.com/en-us/windows/win32/api/sysinfoapi/nf-sysinfoapi-getsysteminfo
+func getSystemInfo() (_SYSTEM_INFO, error) {
+ var si _SYSTEM_INFO
+ _, _, err := procGetSystemInfo.Call(uintptr(unsafe.Pointer(&si)))
+ if err != syscall.Errno(0) {
+ return si, err
+ }
+ return si, nil
+}
+
+// BOOL GetProcessWorkingSetSize(
+// HANDLE hProcess,
+// PSIZE_T lpMinimumWorkingSetSize,
+// PSIZE_T lpMaximumWorkingSetSize
+// );
+// https://docs.microsoft.com/en-us/windows/win32/api/winbase/nf-winbase-getprocessworkingsetsize
+
+func getProcessWorkingSetSize(process uintptr, dwMinWorkingSet *uint64, dwMaxWorkingSet *uint64) error {
+ r1, _, err := syscall.Syscall(procGetProcessWorkingSetSize.Addr(), 3, process, uintptr(unsafe.Pointer(dwMinWorkingSet)), uintptr(unsafe.Pointer(dwMaxWorkingSet)))
+ if r1 == 0 {
+ if err != syscall.Errno(0) {
+ return err
+ }
+ }
+ return nil
+}
+
+// BOOL SetProcessWorkingSetSize(
+// HANDLE hProcess,
+// SIZE_T dwMinimumWorkingSetSize,
+// SIZE_T dwMaximumWorkingSetSize
+// );
+// https://docs.microsoft.com/en-us/windows/win32/api/winbase/nf-winbase-setprocessworkingsetsize
+
+func setProcessWorkingSetSize(process uintptr, dwMinWorkingSet uint64, dwMaxWorkingSet uint64) error {
+ r1, _, err := syscall.Syscall(procSetProcessWorkingSetSize.Addr(), 3, process, uintptr(dwMinWorkingSet), uintptr(dwMaxWorkingSet))
+ if r1 == 0 {
+ if err != syscall.Errno(0) {
+ return err
+ }
+ }
+ return nil
+}
diff --git a/weed/storage/backend/memory_map/os_overloads/file_windows.go b/weed/storage/backend/memory_map/os_overloads/file_windows.go
new file mode 100644
index 000000000..05aa384e2
--- /dev/null
+++ b/weed/storage/backend/memory_map/os_overloads/file_windows.go
@@ -0,0 +1,168 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package os_overloads
+
+import (
+ "os"
+ "syscall"
+
+ "golang.org/x/sys/windows"
+)
+
+func isAbs(path string) (b bool) {
+ v := volumeName(path)
+ if v == "" {
+ return false
+ }
+ path = path[len(v):]
+ if path == "" {
+ return false
+ }
+ return os.IsPathSeparator(path[0])
+}
+
+func volumeName(path string) (v string) {
+ if len(path) < 2 {
+ return ""
+ }
+ // with drive letter
+ c := path[0]
+ if path[1] == ':' &&
+ ('0' <= c && c <= '9' || 'a' <= c && c <= 'z' ||
+ 'A' <= c && c <= 'Z') {
+ return path[:2]
+ }
+ // is it UNC
+ if l := len(path); l >= 5 && os.IsPathSeparator(path[0]) && os.IsPathSeparator(path[1]) &&
+ !os.IsPathSeparator(path[2]) && path[2] != '.' {
+ // first, leading `\\` and next shouldn't be `\`. its server name.
+ for n := 3; n < l-1; n++ {
+ // second, next '\' shouldn't be repeated.
+ if os.IsPathSeparator(path[n]) {
+ n++
+ // third, following something characters. its share name.
+ if !os.IsPathSeparator(path[n]) {
+ if path[n] == '.' {
+ break
+ }
+ for ; n < l; n++ {
+ if os.IsPathSeparator(path[n]) {
+ break
+ }
+ }
+ return path[:n]
+ }
+ break
+ }
+ }
+ }
+ return ""
+}
+
+// fixLongPath returns the extended-length (\\?\-prefixed) form of
+// path when needed, in order to avoid the default 260 character file
+// path limit imposed by Windows. If path is not easily converted to
+// the extended-length form (for example, if path is a relative path
+// or contains .. elements), or is short enough, fixLongPath returns
+// path unmodified.
+//
+// See https://msdn.microsoft.com/en-us/library/windows/desktop/aa365247(v=vs.85).aspx#maxpath
+func fixLongPath(path string) string {
+ // Do nothing (and don't allocate) if the path is "short".
+ // Empirically (at least on the Windows Server 2013 builder),
+ // the kernel is arbitrarily okay with < 248 bytes. That
+ // matches what the docs above say:
+ // "When using an API to create a directory, the specified
+ // path cannot be so long that you cannot append an 8.3 file
+ // name (that is, the directory name cannot exceed MAX_PATH
+ // minus 12)." Since MAX_PATH is 260, 260 - 12 = 248.
+ //
+ // The MSDN docs appear to say that a normal path that is 248 bytes long
+ // will work; empirically the path must be less then 248 bytes long.
+ if len(path) < 248 {
+ // Don't fix. (This is how Go 1.7 and earlier worked,
+ // not automatically generating the \\?\ form)
+ return path
+ }
+
+ // The extended form begins with \\?\, as in
+ // \\?\c:\windows\foo.txt or \\?\UNC\server\share\foo.txt.
+ // The extended form disables evaluation of . and .. path
+ // elements and disables the interpretation of / as equivalent
+ // to \. The conversion here rewrites / to \ and elides
+ // . elements as well as trailing or duplicate separators. For
+ // simplicity it avoids the conversion entirely for relative
+ // paths or paths containing .. elements. For now,
+ // \\server\share paths are not converted to
+ // \\?\UNC\server\share paths because the rules for doing so
+ // are less well-specified.
+ if len(path) >= 2 && path[:2] == `\\` {
+ // Don't canonicalize UNC paths.
+ return path
+ }
+ if !isAbs(path) {
+ // Relative path
+ return path
+ }
+
+ const prefix = `\\?`
+
+ pathbuf := make([]byte, len(prefix)+len(path)+len(`\`))
+ copy(pathbuf, prefix)
+ n := len(path)
+ r, w := 0, len(prefix)
+ for r < n {
+ switch {
+ case os.IsPathSeparator(path[r]):
+ // empty block
+ r++
+ case path[r] == '.' && (r+1 == n || os.IsPathSeparator(path[r+1])):
+ // /./
+ r++
+ case r+1 < n && path[r] == '.' && path[r+1] == '.' && (r+2 == n || os.IsPathSeparator(path[r+2])):
+ // /../ is currently unhandled
+ return path
+ default:
+ pathbuf[w] = '\\'
+ w++
+ for ; r < n && !os.IsPathSeparator(path[r]); r++ {
+ pathbuf[w] = path[r]
+ w++
+ }
+ }
+ }
+ // A drive's root directory needs a trailing \
+ if w == len(`\\?\c:`) {
+ pathbuf[w] = '\\'
+ w++
+ }
+ return string(pathbuf[:w])
+}
+
+// syscallMode returns the syscall-specific mode bits from Go's portable mode bits.
+func syscallMode(i os.FileMode) (o uint32) {
+ o |= uint32(i.Perm())
+ if i&os.ModeSetuid != 0 {
+ o |= syscall.S_ISUID
+ }
+ if i&os.ModeSetgid != 0 {
+ o |= syscall.S_ISGID
+ }
+ if i&os.ModeSticky != 0 {
+ o |= syscall.S_ISVTX
+ }
+ // No mapping for Go's ModeTemporary (plan9 only).
+ return
+}
+
+//If the bool is set to true then the file is opened with the parameters FILE_ATTRIBUTE_TEMPORARY and
+// FILE_FLAG_DELETE_ON_CLOSE
+func OpenFile(name string, flag int, perm os.FileMode, setToTempAndDelete bool) (file *os.File, err error) {
+ r, e := Open(fixLongPath(name), flag|windows.O_CLOEXEC, syscallMode(perm), setToTempAndDelete)
+ if e != nil {
+ return nil, e
+ }
+ return os.NewFile(uintptr(r), name), nil
+}
diff --git a/weed/storage/backend/memory_map/os_overloads/syscall_windows.go b/weed/storage/backend/memory_map/os_overloads/syscall_windows.go
new file mode 100644
index 000000000..081cba431
--- /dev/null
+++ b/weed/storage/backend/memory_map/os_overloads/syscall_windows.go
@@ -0,0 +1,80 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Windows system calls.
+
+package os_overloads
+
+import (
+ "syscall"
+ "unsafe"
+
+ "golang.org/x/sys/windows"
+)
+
+// windows api calls
+
+//sys CreateFile(name *uint16, access uint32, mode uint32, sa *SecurityAttributes, createmode uint32, attrs uint32, templatefile int32) (handle Handle, err error) [failretval==InvalidHandle] = CreateFileW
+
+func makeInheritSa() *syscall.SecurityAttributes {
+ var sa syscall.SecurityAttributes
+ sa.Length = uint32(unsafe.Sizeof(sa))
+ sa.InheritHandle = 1
+ return &sa
+}
+
+// opens the
+func Open(path string, mode int, perm uint32, setToTempAndDelete bool) (fd syscall.Handle, err error) {
+ if len(path) == 0 {
+ return syscall.InvalidHandle, windows.ERROR_FILE_NOT_FOUND
+ }
+ pathp, err := syscall.UTF16PtrFromString(path)
+ if err != nil {
+ return syscall.InvalidHandle, err
+ }
+ var access uint32
+ switch mode & (windows.O_RDONLY | windows.O_WRONLY | windows.O_RDWR) {
+ case windows.O_RDONLY:
+ access = windows.GENERIC_READ
+ case windows.O_WRONLY:
+ access = windows.GENERIC_WRITE
+ case windows.O_RDWR:
+ access = windows.GENERIC_READ | windows.GENERIC_WRITE
+ }
+ if mode&windows.O_CREAT != 0 {
+ access |= windows.GENERIC_WRITE
+ }
+ if mode&windows.O_APPEND != 0 {
+ access &^= windows.GENERIC_WRITE
+ access |= windows.FILE_APPEND_DATA
+ }
+ sharemode := uint32(windows.FILE_SHARE_READ | windows.FILE_SHARE_WRITE)
+ var sa *syscall.SecurityAttributes
+ if mode&windows.O_CLOEXEC == 0 {
+ sa = makeInheritSa()
+ }
+ var createmode uint32
+ switch {
+ case mode&(windows.O_CREAT|windows.O_EXCL) == (windows.O_CREAT | windows.O_EXCL):
+ createmode = windows.CREATE_NEW
+ case mode&(windows.O_CREAT|windows.O_TRUNC) == (windows.O_CREAT | windows.O_TRUNC):
+ createmode = windows.CREATE_ALWAYS
+ case mode&windows.O_CREAT == windows.O_CREAT:
+ createmode = windows.OPEN_ALWAYS
+ case mode&windows.O_TRUNC == windows.O_TRUNC:
+ createmode = windows.TRUNCATE_EXISTING
+ default:
+ createmode = windows.OPEN_EXISTING
+ }
+
+ var h syscall.Handle
+ var e error
+
+ if setToTempAndDelete {
+ h, e = syscall.CreateFile(pathp, access, sharemode, sa, createmode, (windows.FILE_ATTRIBUTE_TEMPORARY | FILE_FLAG_DELETE_ON_CLOSE), 0)
+ } else {
+ h, e = syscall.CreateFile(pathp, access, sharemode, sa, createmode, windows.FILE_ATTRIBUTE_NORMAL, 0)
+ }
+ return h, e
+}
diff --git a/weed/storage/backend/memory_map/os_overloads/types_windows.go b/weed/storage/backend/memory_map/os_overloads/types_windows.go
new file mode 100644
index 000000000..254ba3002
--- /dev/null
+++ b/weed/storage/backend/memory_map/os_overloads/types_windows.go
@@ -0,0 +1,9 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package os_overloads
+
+const (
+ FILE_FLAG_DELETE_ON_CLOSE = 0x04000000
+)
diff --git a/weed/storage/backend/s3_backend/s3_backend.go b/weed/storage/backend/s3_backend/s3_backend.go
new file mode 100644
index 000000000..9f03cfa81
--- /dev/null
+++ b/weed/storage/backend/s3_backend/s3_backend.go
@@ -0,0 +1,177 @@
+package s3_backend
+
+import (
+ "fmt"
+ "io"
+ "os"
+ "strings"
+ "time"
+
+ "github.com/aws/aws-sdk-go/service/s3"
+ "github.com/aws/aws-sdk-go/service/s3/s3iface"
+ "github.com/google/uuid"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+)
+
+func init() {
+ backend.BackendStorageFactories["s3"] = &S3BackendFactory{}
+}
+
+type S3BackendFactory struct {
+}
+
+func (factory *S3BackendFactory) StorageType() backend.StorageType {
+ return backend.StorageType("s3")
+}
+func (factory *S3BackendFactory) BuildStorage(configuration backend.StringProperties, id string) (backend.BackendStorage, error) {
+ return newS3BackendStorage(configuration, id)
+}
+
+type S3BackendStorage struct {
+ id string
+ aws_access_key_id string
+ aws_secret_access_key string
+ region string
+ bucket string
+ conn s3iface.S3API
+}
+
+func newS3BackendStorage(configuration backend.StringProperties, id string) (s *S3BackendStorage, err error) {
+ s = &S3BackendStorage{}
+ s.id = id
+ s.aws_access_key_id = configuration.GetString("aws_access_key_id")
+ s.aws_secret_access_key = configuration.GetString("aws_secret_access_key")
+ s.region = configuration.GetString("region")
+ s.bucket = configuration.GetString("bucket")
+ s.conn, err = createSession(s.aws_access_key_id, s.aws_secret_access_key, s.region)
+
+ glog.V(0).Infof("created backend storage s3.%s for region %s bucket %s", s.id, s.region, s.bucket)
+ return
+}
+
+func (s *S3BackendStorage) ToProperties() map[string]string {
+ m := make(map[string]string)
+ m["aws_access_key_id"] = s.aws_access_key_id
+ m["aws_secret_access_key"] = s.aws_secret_access_key
+ m["region"] = s.region
+ m["bucket"] = s.bucket
+ return m
+}
+
+func (s *S3BackendStorage) NewStorageFile(key string, tierInfo *volume_server_pb.VolumeInfo) backend.BackendStorageFile {
+ if strings.HasPrefix(key, "/") {
+ key = key[1:]
+ }
+
+ f := &S3BackendStorageFile{
+ backendStorage: s,
+ key: key,
+ tierInfo: tierInfo,
+ }
+
+ return f
+}
+
+func (s *S3BackendStorage) CopyFile(f *os.File, attributes map[string]string, fn func(progressed int64, percentage float32) error) (key string, size int64, err error) {
+ randomUuid, _ := uuid.NewRandom()
+ key = randomUuid.String()
+
+ glog.V(1).Infof("copying dat file of %s to remote s3.%s as %s", f.Name(), s.id, key)
+
+ size, err = uploadToS3(s.conn, f.Name(), s.bucket, key, attributes, fn)
+
+ return
+}
+
+func (s *S3BackendStorage) DownloadFile(fileName string, key string, fn func(progressed int64, percentage float32) error) (size int64, err error) {
+
+ glog.V(1).Infof("download dat file of %s from remote s3.%s as %s", fileName, s.id, key)
+
+ size, err = downloadFromS3(s.conn, fileName, s.bucket, key, fn)
+
+ return
+}
+
+func (s *S3BackendStorage) DeleteFile(key string) (err error) {
+
+ glog.V(1).Infof("delete dat file %s from remote", key)
+
+ err = deleteFromS3(s.conn, s.bucket, key)
+
+ return
+}
+
+type S3BackendStorageFile struct {
+ backendStorage *S3BackendStorage
+ key string
+ tierInfo *volume_server_pb.VolumeInfo
+}
+
+func (s3backendStorageFile S3BackendStorageFile) ReadAt(p []byte, off int64) (n int, err error) {
+
+ bytesRange := fmt.Sprintf("bytes=%d-%d", off, off+int64(len(p))-1)
+
+ // glog.V(0).Infof("read %s %s", s3backendStorageFile.key, bytesRange)
+
+ getObjectOutput, getObjectErr := s3backendStorageFile.backendStorage.conn.GetObject(&s3.GetObjectInput{
+ Bucket: &s3backendStorageFile.backendStorage.bucket,
+ Key: &s3backendStorageFile.key,
+ Range: &bytesRange,
+ })
+
+ if getObjectErr != nil {
+ return 0, fmt.Errorf("bucket %s GetObject %s: %v", s3backendStorageFile.backendStorage.bucket, s3backendStorageFile.key, getObjectErr)
+ }
+ defer getObjectOutput.Body.Close()
+
+ glog.V(4).Infof("read %s %s", s3backendStorageFile.key, bytesRange)
+ glog.V(4).Infof("content range: %s, contentLength: %d", *getObjectOutput.ContentRange, *getObjectOutput.ContentLength)
+
+ for {
+ if n, err = getObjectOutput.Body.Read(p); err == nil && n < len(p) {
+ p = p[n:]
+ } else {
+ break
+ }
+ }
+
+ if err == io.EOF {
+ err = nil
+ }
+
+ return
+}
+
+func (s3backendStorageFile S3BackendStorageFile) WriteAt(p []byte, off int64) (n int, err error) {
+ panic("not implemented")
+}
+
+func (s3backendStorageFile S3BackendStorageFile) Truncate(off int64) error {
+ panic("not implemented")
+}
+
+func (s3backendStorageFile S3BackendStorageFile) Close() error {
+ return nil
+}
+
+func (s3backendStorageFile S3BackendStorageFile) GetStat() (datSize int64, modTime time.Time, err error) {
+
+ files := s3backendStorageFile.tierInfo.GetFiles()
+
+ if len(files) == 0 {
+ err = fmt.Errorf("remote file info not found")
+ return
+ }
+
+ datSize = int64(files[0].FileSize)
+ modTime = time.Unix(int64(files[0].ModifiedTime), 0)
+
+ return
+}
+
+func (s3backendStorageFile S3BackendStorageFile) Name() string {
+ return s3backendStorageFile.key
+}
diff --git a/weed/storage/backend/s3_backend/s3_download.go b/weed/storage/backend/s3_backend/s3_download.go
new file mode 100644
index 000000000..dbc28446a
--- /dev/null
+++ b/weed/storage/backend/s3_backend/s3_download.go
@@ -0,0 +1,98 @@
+package s3_backend
+
+import (
+ "fmt"
+ "os"
+ "sync/atomic"
+
+ "github.com/aws/aws-sdk-go/aws"
+ "github.com/aws/aws-sdk-go/service/s3"
+ "github.com/aws/aws-sdk-go/service/s3/s3iface"
+ "github.com/aws/aws-sdk-go/service/s3/s3manager"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+)
+
+func downloadFromS3(sess s3iface.S3API, destFileName string, sourceBucket string, sourceKey string,
+ fn func(progressed int64, percentage float32) error) (fileSize int64, err error) {
+
+ fileSize, err = getFileSize(sess, sourceBucket, sourceKey)
+ if err != nil {
+ return
+ }
+
+ //open the file
+ f, err := os.OpenFile(destFileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if err != nil {
+ return 0, fmt.Errorf("failed to open file %q, %v", destFileName, err)
+ }
+ defer f.Close()
+
+ // Create a downloader with the session and custom options
+ downloader := s3manager.NewDownloaderWithClient(sess, func(u *s3manager.Downloader) {
+ u.PartSize = int64(64 * 1024 * 1024)
+ u.Concurrency = 5
+ })
+
+ fileWriter := &s3DownloadProgressedWriter{
+ fp: f,
+ size: fileSize,
+ written: 0,
+ fn: fn,
+ }
+
+ // Download the file from S3.
+ fileSize, err = downloader.Download(fileWriter, &s3.GetObjectInput{
+ Bucket: aws.String(sourceBucket),
+ Key: aws.String(sourceKey),
+ })
+ if err != nil {
+ return fileSize, fmt.Errorf("failed to download file %s: %v", destFileName, err)
+ }
+
+ glog.V(1).Infof("downloaded file %s\n", destFileName)
+
+ return
+}
+
+// adapted from https://github.com/aws/aws-sdk-go/pull/1868
+// and https://petersouter.xyz/s3-download-progress-bar-in-golang/
+type s3DownloadProgressedWriter struct {
+ fp *os.File
+ size int64
+ written int64
+ fn func(progressed int64, percentage float32) error
+}
+
+func (w *s3DownloadProgressedWriter) WriteAt(p []byte, off int64) (int, error) {
+ n, err := w.fp.WriteAt(p, off)
+ if err != nil {
+ return n, err
+ }
+
+ // Got the length have read( or means has uploaded), and you can construct your message
+ atomic.AddInt64(&w.written, int64(n))
+
+ if w.fn != nil {
+ written := w.written
+ if err := w.fn(written, float32(written*100)/float32(w.size)); err != nil {
+ return n, err
+ }
+ }
+
+ return n, err
+}
+
+func getFileSize(svc s3iface.S3API, bucket string, key string) (filesize int64, error error) {
+ params := &s3.HeadObjectInput{
+ Bucket: aws.String(bucket),
+ Key: aws.String(key),
+ }
+
+ resp, err := svc.HeadObject(params)
+ if err != nil {
+ return 0, err
+ }
+
+ return *resp.ContentLength, nil
+}
diff --git a/weed/storage/backend/s3_backend/s3_sessions.go b/weed/storage/backend/s3_backend/s3_sessions.go
new file mode 100644
index 000000000..5fdbcb66b
--- /dev/null
+++ b/weed/storage/backend/s3_backend/s3_sessions.go
@@ -0,0 +1,62 @@
+package s3_backend
+
+import (
+ "fmt"
+ "sync"
+
+ "github.com/aws/aws-sdk-go/aws"
+ "github.com/aws/aws-sdk-go/aws/credentials"
+ "github.com/aws/aws-sdk-go/aws/session"
+ "github.com/aws/aws-sdk-go/service/s3"
+ "github.com/aws/aws-sdk-go/service/s3/s3iface"
+)
+
+var (
+ s3Sessions = make(map[string]s3iface.S3API)
+ sessionsLock sync.RWMutex
+)
+
+func getSession(region string) (s3iface.S3API, bool) {
+ sessionsLock.RLock()
+ defer sessionsLock.RUnlock()
+
+ sess, found := s3Sessions[region]
+ return sess, found
+}
+
+func createSession(awsAccessKeyId, awsSecretAccessKey, region string) (s3iface.S3API, error) {
+
+ sessionsLock.Lock()
+ defer sessionsLock.Unlock()
+
+ if t, found := s3Sessions[region]; found {
+ return t, nil
+ }
+
+ config := &aws.Config{
+ Region: aws.String(region),
+ }
+ if awsAccessKeyId != "" && awsSecretAccessKey != "" {
+ config.Credentials = credentials.NewStaticCredentials(awsAccessKeyId, awsSecretAccessKey, "")
+ }
+
+ sess, err := session.NewSession(config)
+ if err != nil {
+ return nil, fmt.Errorf("create aws session in region %s: %v", region, err)
+ }
+
+ t := s3.New(sess)
+
+ s3Sessions[region] = t
+
+ return t, nil
+
+}
+
+func deleteFromS3(sess s3iface.S3API, sourceBucket string, sourceKey string) (err error) {
+ _, err = sess.DeleteObject(&s3.DeleteObjectInput{
+ Bucket: aws.String(sourceBucket),
+ Key: aws.String(sourceKey),
+ })
+ return err
+}
diff --git a/weed/storage/backend/s3_backend/s3_upload.go b/weed/storage/backend/s3_backend/s3_upload.go
new file mode 100644
index 000000000..500a85590
--- /dev/null
+++ b/weed/storage/backend/s3_backend/s3_upload.go
@@ -0,0 +1,114 @@
+package s3_backend
+
+import (
+ "fmt"
+ "os"
+ "sync/atomic"
+
+ "github.com/aws/aws-sdk-go/aws"
+ "github.com/aws/aws-sdk-go/service/s3/s3iface"
+ "github.com/aws/aws-sdk-go/service/s3/s3manager"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+)
+
+func uploadToS3(sess s3iface.S3API, filename string, destBucket string, destKey string,
+ attributes map[string]string,
+ fn func(progressed int64, percentage float32) error) (fileSize int64, err error) {
+
+ //open the file
+ f, err := os.Open(filename)
+ if err != nil {
+ return 0, fmt.Errorf("failed to open file %q, %v", filename, err)
+ }
+ defer f.Close()
+
+ info, err := f.Stat()
+ if err != nil {
+ return 0, fmt.Errorf("failed to stat file %q, %v", filename, err)
+ }
+
+ fileSize = info.Size()
+
+ partSize := int64(64 * 1024 * 1024) // The minimum/default allowed part size is 5MB
+ for partSize*1000 < fileSize {
+ partSize *= 4
+ }
+
+ // Create an uploader with the session and custom options
+ uploader := s3manager.NewUploaderWithClient(sess, func(u *s3manager.Uploader) {
+ u.PartSize = partSize
+ u.Concurrency = 5
+ })
+
+ fileReader := &s3UploadProgressedReader{
+ fp: f,
+ size: fileSize,
+ read: -fileSize,
+ fn: fn,
+ }
+
+ // process tagging
+ tags := ""
+ for k, v := range attributes {
+ if len(tags) > 0 {
+ tags = tags + "&"
+ }
+ tags = tags + k + "=" + v
+ }
+
+ // Upload the file to S3.
+ var result *s3manager.UploadOutput
+ result, err = uploader.Upload(&s3manager.UploadInput{
+ Bucket: aws.String(destBucket),
+ Key: aws.String(destKey),
+ Body: fileReader,
+ ACL: aws.String("private"),
+ ServerSideEncryption: aws.String("AES256"),
+ StorageClass: aws.String("STANDARD_IA"),
+ Tagging: aws.String(tags),
+ })
+
+ //in case it fails to upload
+ if err != nil {
+ return 0, fmt.Errorf("failed to upload file %s: %v", filename, err)
+ }
+ glog.V(1).Infof("file %s uploaded to %s\n", filename, result.Location)
+
+ return
+}
+
+// adapted from https://github.com/aws/aws-sdk-go/pull/1868
+type s3UploadProgressedReader struct {
+ fp *os.File
+ size int64
+ read int64
+ fn func(progressed int64, percentage float32) error
+}
+
+func (r *s3UploadProgressedReader) Read(p []byte) (int, error) {
+ return r.fp.Read(p)
+}
+
+func (r *s3UploadProgressedReader) ReadAt(p []byte, off int64) (int, error) {
+ n, err := r.fp.ReadAt(p, off)
+ if err != nil {
+ return n, err
+ }
+
+ // Got the length have read( or means has uploaded), and you can construct your message
+ atomic.AddInt64(&r.read, int64(n))
+
+ if r.fn != nil {
+ read := r.read
+ if err := r.fn(read, float32(read*100)/float32(r.size)); err != nil {
+ return n, err
+ }
+ }
+
+ return n, err
+}
+
+func (r *s3UploadProgressedReader) Seek(offset int64, whence int) (int64, error) {
+ return r.fp.Seek(offset, whence)
+}
diff --git a/weed/storage/disk_location.go b/weed/storage/disk_location.go
index e61623fc7..e116fc715 100644
--- a/weed/storage/disk_location.go
+++ b/weed/storage/disk_location.go
@@ -17,7 +17,7 @@ type DiskLocation struct {
Directory string
MaxVolumeCount int
volumes map[needle.VolumeId]*Volume
- sync.RWMutex
+ volumesLock sync.RWMutex
// erasure coding
ecVolumes map[needle.VolumeId]*erasure_coding.EcVolume
@@ -33,8 +33,8 @@ func NewDiskLocation(dir string, maxVolumeCount int) *DiskLocation {
func (l *DiskLocation) volumeIdFromPath(dir os.FileInfo) (needle.VolumeId, string, error) {
name := dir.Name()
- if !dir.IsDir() && strings.HasSuffix(name, ".dat") {
- base := name[:len(name)-len(".dat")]
+ if !dir.IsDir() && strings.HasSuffix(name, ".idx") {
+ base := name[:len(name)-len(".idx")]
collection, volumeId, err := parseCollectionVolumeId(base)
return volumeId, collection, err
}
@@ -53,17 +53,17 @@ func parseCollectionVolumeId(base string) (collection string, vid needle.VolumeI
func (l *DiskLocation) loadExistingVolume(fileInfo os.FileInfo, needleMapKind NeedleMapType) {
name := fileInfo.Name()
- if !fileInfo.IsDir() && strings.HasSuffix(name, ".dat") {
+ if !fileInfo.IsDir() && strings.HasSuffix(name, ".idx") {
vid, collection, err := l.volumeIdFromPath(fileInfo)
if err == nil {
- l.RLock()
+ l.volumesLock.RLock()
_, found := l.volumes[vid]
- l.RUnlock()
+ l.volumesLock.RUnlock()
if !found {
- if v, e := NewVolume(l.Directory, collection, vid, needleMapKind, nil, nil, 0); e == nil {
- l.Lock()
+ if v, e := NewVolume(l.Directory, collection, vid, needleMapKind, nil, nil, 0, 0); e == nil {
+ l.volumesLock.Lock()
l.volumes[vid] = v
- l.Unlock()
+ l.volumesLock.Unlock()
size, _, _ := v.FileStat()
glog.V(0).Infof("data file %s, replicaPlacement=%s v=%d size=%d ttl=%s",
l.Directory+"/"+name, v.ReplicaPlacement, v.Version(), size, v.Ttl.String())
@@ -71,7 +71,6 @@ func (l *DiskLocation) loadExistingVolume(fileInfo os.FileInfo, needleMapKind Ne
} else {
glog.V(0).Infof("new volume %s error %s", name, e)
}
-
}
}
}
@@ -115,29 +114,46 @@ func (l *DiskLocation) loadExistingVolumes(needleMapKind NeedleMapType) {
func (l *DiskLocation) DeleteCollectionFromDiskLocation(collection string) (e error) {
- l.Lock()
- for k, v := range l.volumes {
- if v.Collection == collection {
- e = l.deleteVolumeById(k)
- if e != nil {
- l.Unlock()
- return
- }
- }
- }
- l.Unlock()
+ l.volumesLock.Lock()
+ delVolsMap := l.unmountVolumeByCollection(collection)
+ l.volumesLock.Unlock()
l.ecVolumesLock.Lock()
- for k, v := range l.ecVolumes {
- if v.Collection == collection {
- e = l.deleteEcVolumeById(k)
- if e != nil {
- l.ecVolumesLock.Unlock()
- return
+ delEcVolsMap := l.unmountEcVolumeByCollection(collection)
+ l.ecVolumesLock.Unlock()
+
+ errChain := make(chan error, 2)
+ var wg sync.WaitGroup
+ wg.Add(2)
+ go func() {
+ for _, v := range delVolsMap {
+ if err := v.Destroy(); err != nil {
+ errChain <- err
}
}
+ wg.Done()
+ }()
+
+ go func() {
+ for _, v := range delEcVolsMap {
+ v.Destroy()
+ }
+ wg.Done()
+ }()
+
+ go func() {
+ wg.Wait()
+ close(errChain)
+ }()
+
+ errBuilder := strings.Builder{}
+ for err := range errChain {
+ errBuilder.WriteString(err.Error())
+ errBuilder.WriteString("; ")
+ }
+ if errBuilder.Len() > 0 {
+ e = fmt.Errorf(errBuilder.String())
}
- l.ecVolumesLock.Unlock()
return
}
@@ -170,8 +186,8 @@ func (l *DiskLocation) LoadVolume(vid needle.VolumeId, needleMapKind NeedleMapTy
}
func (l *DiskLocation) DeleteVolume(vid needle.VolumeId) error {
- l.Lock()
- defer l.Unlock()
+ l.volumesLock.Lock()
+ defer l.volumesLock.Unlock()
_, ok := l.volumes[vid]
if !ok {
@@ -181,8 +197,8 @@ func (l *DiskLocation) DeleteVolume(vid needle.VolumeId) error {
}
func (l *DiskLocation) UnloadVolume(vid needle.VolumeId) error {
- l.Lock()
- defer l.Unlock()
+ l.volumesLock.Lock()
+ defer l.volumesLock.Unlock()
v, ok := l.volumes[vid]
if !ok {
@@ -193,34 +209,48 @@ func (l *DiskLocation) UnloadVolume(vid needle.VolumeId) error {
return nil
}
+func (l *DiskLocation) unmountVolumeByCollection(collectionName string) map[needle.VolumeId]*Volume {
+ deltaVols := make(map[needle.VolumeId]*Volume, 0)
+ for k, v := range l.volumes {
+ if v.Collection == collectionName && !v.isCompacting {
+ deltaVols[k] = v
+ }
+ }
+
+ for k, _ := range deltaVols {
+ delete(l.volumes, k)
+ }
+ return deltaVols
+}
+
func (l *DiskLocation) SetVolume(vid needle.VolumeId, volume *Volume) {
- l.Lock()
- defer l.Unlock()
+ l.volumesLock.Lock()
+ defer l.volumesLock.Unlock()
l.volumes[vid] = volume
}
func (l *DiskLocation) FindVolume(vid needle.VolumeId) (*Volume, bool) {
- l.RLock()
- defer l.RUnlock()
+ l.volumesLock.RLock()
+ defer l.volumesLock.RUnlock()
v, ok := l.volumes[vid]
return v, ok
}
func (l *DiskLocation) VolumesLen() int {
- l.RLock()
- defer l.RUnlock()
+ l.volumesLock.RLock()
+ defer l.volumesLock.RUnlock()
return len(l.volumes)
}
func (l *DiskLocation) Close() {
- l.Lock()
+ l.volumesLock.Lock()
for _, v := range l.volumes {
v.Close()
}
- l.Unlock()
+ l.volumesLock.Unlock()
l.ecVolumesLock.Lock()
for _, ecVolume := range l.ecVolumes {
diff --git a/weed/storage/disk_location_ec.go b/weed/storage/disk_location_ec.go
index ba0824c6d..f6c44e966 100644
--- a/weed/storage/disk_location_ec.go
+++ b/weed/storage/disk_location_ec.go
@@ -169,3 +169,17 @@ func (l *DiskLocation) deleteEcVolumeById(vid needle.VolumeId) (e error) {
delete(l.ecVolumes, vid)
return
}
+
+func (l *DiskLocation) unmountEcVolumeByCollection(collectionName string) map[needle.VolumeId]*erasure_coding.EcVolume {
+ deltaVols := make(map[needle.VolumeId]*erasure_coding.EcVolume, 0)
+ for k, v := range l.ecVolumes {
+ if v.Collection == collectionName {
+ deltaVols[k] = v
+ }
+ }
+
+ for k, _ := range deltaVols {
+ delete(l.ecVolumes, k)
+ }
+ return deltaVols
+}
diff --git a/weed/storage/erasure_coding/ec_decoder.go b/weed/storage/erasure_coding/ec_decoder.go
new file mode 100644
index 000000000..ae77cee3f
--- /dev/null
+++ b/weed/storage/erasure_coding/ec_decoder.go
@@ -0,0 +1,198 @@
+package erasure_coding
+
+import (
+ "fmt"
+ "io"
+ "os"
+
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/idx"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+ "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+// write .idx file from .ecx and .ecj files
+func WriteIdxFileFromEcIndex(baseFileName string) (err error) {
+
+ ecxFile, openErr := os.OpenFile(baseFileName+".ecx", os.O_RDONLY, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open ec index %s.ecx: %v", baseFileName, openErr)
+ }
+ defer ecxFile.Close()
+
+ idxFile, openErr := os.OpenFile(baseFileName+".idx", os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open %s.idx: %v", baseFileName, openErr)
+ }
+ defer idxFile.Close()
+
+ io.Copy(idxFile, ecxFile)
+
+ err = iterateEcjFile(baseFileName, func(key types.NeedleId) error {
+
+ bytes := needle_map.ToBytes(key, types.Offset{}, types.TombstoneFileSize)
+ idxFile.Write(bytes)
+
+ return nil
+ })
+
+ return err
+}
+
+// FindDatFileSize calculate .dat file size from max offset entry
+// there may be extra deletions after that entry
+// but they are deletions anyway
+func FindDatFileSize(baseFileName string) (datSize int64, err error) {
+
+ version, err := readEcVolumeVersion(baseFileName)
+ if err != nil {
+ return 0, fmt.Errorf("read ec volume %s version: %v", baseFileName, err)
+ }
+
+ err = iterateEcxFile(baseFileName, func(key types.NeedleId, offset types.Offset, size uint32) error {
+
+ if size == types.TombstoneFileSize {
+ return nil
+ }
+
+ entryStopOffset := offset.ToAcutalOffset() + needle.GetActualSize(size, version)
+ if datSize < entryStopOffset {
+ datSize = entryStopOffset
+ }
+
+ return nil
+ })
+
+ return
+}
+
+func readEcVolumeVersion(baseFileName string) (version needle.Version, err error) {
+
+ // find volume version
+ datFile, err := os.OpenFile(baseFileName+".ec00", os.O_RDONLY, 0644)
+ if err != nil {
+ return 0, fmt.Errorf("open ec volume %s superblock: %v", baseFileName, err)
+ }
+ datBackend := backend.NewDiskFile(datFile)
+
+ superBlock, err := super_block.ReadSuperBlock(datBackend)
+ datBackend.Close()
+ if err != nil {
+ return 0, fmt.Errorf("read ec volume %s superblock: %v", baseFileName, err)
+ }
+
+ return superBlock.Version, nil
+
+}
+
+func iterateEcxFile(baseFileName string, processNeedleFn func(key types.NeedleId, offset types.Offset, size uint32) error) error {
+ ecxFile, openErr := os.OpenFile(baseFileName+".ecx", os.O_RDONLY, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open ec index %s.ecx: %v", baseFileName, openErr)
+ }
+ defer ecxFile.Close()
+
+ buf := make([]byte, types.NeedleMapEntrySize)
+ for {
+ n, err := ecxFile.Read(buf)
+ if n != types.NeedleMapEntrySize {
+ if err == io.EOF {
+ return nil
+ }
+ return err
+ }
+ key, offset, size := idx.IdxFileEntry(buf)
+ if processNeedleFn != nil {
+ err = processNeedleFn(key, offset, size)
+ }
+ if err != nil {
+ if err != io.EOF {
+ return err
+ }
+ return nil
+ }
+ }
+
+}
+
+func iterateEcjFile(baseFileName string, processNeedleFn func(key types.NeedleId) error) error {
+ ecjFile, openErr := os.OpenFile(baseFileName+".ecj", os.O_RDONLY, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open ec index %s.ecx: %v", baseFileName, openErr)
+ }
+ defer ecjFile.Close()
+
+ buf := make([]byte, types.NeedleIdSize)
+ for {
+ n, err := ecjFile.Read(buf)
+ if n != types.NeedleIdSize {
+ if err == io.EOF {
+ return nil
+ }
+ return err
+ }
+ if processNeedleFn != nil {
+ err = processNeedleFn(types.BytesToNeedleId(buf))
+ }
+ if err != nil {
+ if err == io.EOF {
+ return nil
+ }
+ return err
+ }
+ }
+
+}
+
+// WriteDatFile generates .dat from from .ec00 ~ .ec09 files
+func WriteDatFile(baseFileName string, datFileSize int64) error {
+
+ datFile, openErr := os.OpenFile(baseFileName+".dat", os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot write volume %s.dat: %v", baseFileName, openErr)
+ }
+ defer datFile.Close()
+
+ inputFiles := make([]*os.File, DataShardsCount)
+
+ for shardId := 0; shardId < DataShardsCount; shardId++ {
+ shardFileName := baseFileName + ToExt(shardId)
+ inputFiles[shardId], openErr = os.OpenFile(shardFileName, os.O_RDONLY, 0)
+ if openErr != nil {
+ return openErr
+ }
+ defer inputFiles[shardId].Close()
+ }
+
+ for datFileSize >= DataShardsCount*ErasureCodingLargeBlockSize {
+ for shardId := 0; shardId < DataShardsCount; shardId++ {
+ w, err := io.CopyN(datFile, inputFiles[shardId], ErasureCodingLargeBlockSize)
+ if w != ErasureCodingLargeBlockSize {
+ return fmt.Errorf("copy %s large block %d: %v", baseFileName, shardId, err)
+ }
+ datFileSize -= ErasureCodingLargeBlockSize
+ }
+ }
+
+ for datFileSize > 0 {
+ for shardId := 0; shardId < DataShardsCount; shardId++ {
+ toRead := min(datFileSize, ErasureCodingSmallBlockSize)
+ w, err := io.CopyN(datFile, inputFiles[shardId], toRead)
+ if w != toRead {
+ return fmt.Errorf("copy %s small block %d: %v", baseFileName, shardId, err)
+ }
+ datFileSize -= toRead
+ }
+ }
+
+ return nil
+}
+
+func min(x, y int64) int64 {
+ if x > y {
+ return y
+ }
+ return x
+}
diff --git a/weed/storage/erasure_coding/ec_encoder.go b/weed/storage/erasure_coding/ec_encoder.go
index 97010a1ed..eeb384b91 100644
--- a/weed/storage/erasure_coding/ec_encoder.go
+++ b/weed/storage/erasure_coding/ec_encoder.go
@@ -21,16 +21,16 @@ const (
ErasureCodingSmallBlockSize = 1024 * 1024 // 1MB
)
-// WriteSortedEcxFile generates .ecx file from existing .idx file
+// WriteSortedFileFromIdx generates .ecx file from existing .idx file
// all keys are sorted in ascending order
-func WriteSortedEcxFile(baseFileName string) (e error) {
+func WriteSortedFileFromIdx(baseFileName string, ext string) (e error) {
cm, err := readCompactMap(baseFileName)
if err != nil {
return fmt.Errorf("readCompactMap: %v", err)
}
- ecxFile, err := os.OpenFile(baseFileName+".ecx", os.O_TRUNC|os.O_CREATE|os.O_WRONLY, 0644)
+ ecxFile, err := os.OpenFile(baseFileName+ext, os.O_TRUNC|os.O_CREATE|os.O_WRONLY, 0644)
if err != nil {
return fmt.Errorf("failed to open ecx file: %v", err)
}
@@ -43,13 +43,13 @@ func WriteSortedEcxFile(baseFileName string) (e error) {
})
if err != nil {
- return fmt.Errorf("failed to visit ecx file: %v", err)
+ return fmt.Errorf("failed to visit idx file: %v", err)
}
return nil
}
-// WriteEcFiles generates .ec01 ~ .ec14 files
+// WriteEcFiles generates .ec00 ~ .ec13 files
func WriteEcFiles(baseFileName string) error {
return generateEcFiles(baseFileName, 256*1024, ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize)
}
diff --git a/weed/storage/erasure_coding/ec_test.go b/weed/storage/erasure_coding/ec_test.go
index 57df09525..0e4aaa27c 100644
--- a/weed/storage/erasure_coding/ec_test.go
+++ b/weed/storage/erasure_coding/ec_test.go
@@ -26,14 +26,14 @@ func TestEncodingDecoding(t *testing.T) {
t.Logf("generateEcFiles: %v", err)
}
- err = WriteSortedEcxFile(baseFileName)
+ err = WriteSortedFileFromIdx(baseFileName, ".ecx")
if err != nil {
- t.Logf("WriteSortedEcxFile: %v", err)
+ t.Logf("WriteSortedFileFromIdx: %v", err)
}
err = validateFiles(baseFileName)
if err != nil {
- t.Logf("WriteSortedEcxFile: %v", err)
+ t.Logf("WriteSortedFileFromIdx: %v", err)
}
removeGeneratedFiles(baseFileName)
diff --git a/weed/storage/erasure_coding/ec_volume.go b/weed/storage/erasure_coding/ec_volume.go
index bcae164ca..579f037fb 100644
--- a/weed/storage/erasure_coding/ec_volume.go
+++ b/weed/storage/erasure_coding/ec_volume.go
@@ -9,7 +9,9 @@ import (
"sync"
"time"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/types"
@@ -56,6 +58,14 @@ func NewEcVolume(dir string, collection string, vid needle.VolumeId) (ev *EcVolu
return nil, fmt.Errorf("cannot open ec volume journal %s.ecj: %v", baseFileName, err)
}
+ // read volume info
+ ev.Version = needle.Version3
+ if volumeInfo, found := pb.MaybeLoadVolumeInfo(baseFileName + ".vif"); found {
+ ev.Version = needle.Version(volumeInfo.Version)
+ } else {
+ pb.SaveVolumeInfo(baseFileName+".vif", &volume_server_pb.VolumeInfo{Version: uint32(ev.Version)})
+ }
+
ev.ShardLocations = make(map[ShardId][]string)
return
@@ -126,6 +136,7 @@ func (ev *EcVolume) Destroy() {
}
os.Remove(ev.FileName() + ".ecx")
os.Remove(ev.FileName() + ".ecj")
+ os.Remove(ev.FileName() + ".vif")
}
func (ev *EcVolume) FileName() string {
@@ -186,10 +197,10 @@ func (ev *EcVolume) LocateEcShardNeedle(needleId types.NeedleId, version needle.
}
func (ev *EcVolume) FindNeedleFromEcx(needleId types.NeedleId) (offset types.Offset, size uint32, err error) {
- return searchNeedleFromEcx(ev.ecxFile, ev.ecxFileSize, needleId, nil)
+ return SearchNeedleFromSortedIndex(ev.ecxFile, ev.ecxFileSize, needleId, nil)
}
-func searchNeedleFromEcx(ecxFile *os.File, ecxFileSize int64, needleId types.NeedleId, processNeedleFn func(file *os.File, offset int64) error) (offset types.Offset, size uint32, err error) {
+func SearchNeedleFromSortedIndex(ecxFile *os.File, ecxFileSize int64, needleId types.NeedleId, processNeedleFn func(file *os.File, offset int64) error) (offset types.Offset, size uint32, err error) {
var key types.NeedleId
buf := make([]byte, types.NeedleMapEntrySize)
l, h := int64(0), ecxFileSize/types.NeedleMapEntrySize
diff --git a/weed/storage/erasure_coding/ec_volume_delete.go b/weed/storage/erasure_coding/ec_volume_delete.go
index 04102ec9e..822a9e923 100644
--- a/weed/storage/erasure_coding/ec_volume_delete.go
+++ b/weed/storage/erasure_coding/ec_volume_delete.go
@@ -10,15 +10,15 @@ import (
)
var (
- markNeedleDeleted = func(file *os.File, offset int64) error {
+ MarkNeedleDeleted = func(file *os.File, offset int64) error {
b := make([]byte, types.SizeSize)
util.Uint32toBytes(b, types.TombstoneFileSize)
n, err := file.WriteAt(b, offset+types.NeedleIdSize+types.OffsetSize)
if err != nil {
- return fmt.Errorf("ecx write error: %v", err)
+ return fmt.Errorf("sorted needle write error: %v", err)
}
if n != types.SizeSize {
- return fmt.Errorf("ecx written %d bytes, expecting %d", n, types.SizeSize)
+ return fmt.Errorf("sorted needle written %d bytes, expecting %d", n, types.SizeSize)
}
return nil
}
@@ -26,7 +26,7 @@ var (
func (ev *EcVolume) DeleteNeedleFromEcx(needleId types.NeedleId) (err error) {
- _, _, err = searchNeedleFromEcx(ev.ecxFile, ev.ecxFileSize, needleId, markNeedleDeleted)
+ _, _, err = SearchNeedleFromSortedIndex(ev.ecxFile, ev.ecxFileSize, needleId, MarkNeedleDeleted)
if err != nil {
if err == NotFoundError {
@@ -81,7 +81,7 @@ func RebuildEcxFile(baseFileName string) error {
needleId := types.BytesToNeedleId(buf)
- _, _, err = searchNeedleFromEcx(ecxFile, ecxFileSize, needleId, markNeedleDeleted)
+ _, _, err = SearchNeedleFromSortedIndex(ecxFile, ecxFileSize, needleId, MarkNeedleDeleted)
if err != nil && err != NotFoundError {
ecxFile.Close()
diff --git a/weed/storage/erasure_coding/ec_volume_info.go b/weed/storage/erasure_coding/ec_volume_info.go
index c9e85c662..8ff65bb0f 100644
--- a/weed/storage/erasure_coding/ec_volume_info.go
+++ b/weed/storage/erasure_coding/ec_volume_info.go
@@ -81,6 +81,15 @@ func (b ShardBits) ShardIds() (ret []ShardId) {
return
}
+func (b ShardBits) ToUint32Slice() (ret []uint32) {
+ for i := uint32(0); i < TotalShardsCount; i++ {
+ if b.HasShardId(ShardId(i)) {
+ ret = append(ret, i)
+ }
+ }
+ return
+}
+
func (b ShardBits) ShardIdCount() (count int) {
for count = 0; b > 0; count++ {
b &= b - 1
@@ -95,3 +104,10 @@ func (b ShardBits) Minus(other ShardBits) ShardBits {
func (b ShardBits) Plus(other ShardBits) ShardBits {
return b | other
}
+
+func (b ShardBits) MinusParityShards() ShardBits {
+ for i := DataShardsCount; i < TotalShardsCount; i++ {
+ b = b.RemoveShardId(ShardId(i))
+ }
+ return b
+}
diff --git a/weed/storage/needle/needle_read_write.go b/weed/storage/needle/needle_read_write.go
index 75aefdf16..7f8aa4823 100644
--- a/weed/storage/needle/needle_read_write.go
+++ b/weed/storage/needle/needle_read_write.go
@@ -4,11 +4,10 @@ import (
"errors"
"fmt"
"io"
- "os"
-
"math"
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -29,39 +28,25 @@ func (n *Needle) DiskSize(version Version) int64 {
return GetActualSize(n.Size, version)
}
-func (n *Needle) Append(w *os.File, version Version) (offset uint64, size uint32, actualSize int64, err error) {
- if end, e := w.Seek(0, io.SeekEnd); e == nil {
- defer func(w *os.File, off int64) {
- if err != nil {
- if te := w.Truncate(end); te != nil {
- glog.V(0).Infof("Failed to truncate %s back to %d with error: %v", w.Name(), end, te)
- }
- }
- }(w, end)
- offset = uint64(end)
- } else {
- err = fmt.Errorf("Cannot Read Current Volume Position: %v", e)
- return
- }
+func (n *Needle) prepareWriteBuffer(version Version) ([]byte, uint32, int64, error) {
+
+ writeBytes := make([]byte, 0)
+
switch version {
case Version1:
header := make([]byte, NeedleHeaderSize)
CookieToBytes(header[0:CookieSize], n.Cookie)
NeedleIdToBytes(header[CookieSize:CookieSize+NeedleIdSize], n.Id)
n.Size = uint32(len(n.Data))
- size = n.Size
util.Uint32toBytes(header[CookieSize+NeedleIdSize:CookieSize+NeedleIdSize+SizeSize], n.Size)
- if _, err = w.Write(header); err != nil {
- return
- }
- if _, err = w.Write(n.Data); err != nil {
- return
- }
- actualSize = NeedleHeaderSize + int64(n.Size)
+ size := n.Size
+ actualSize := NeedleHeaderSize + int64(n.Size)
+ writeBytes = append(writeBytes, header...)
+ writeBytes = append(writeBytes, n.Data...)
padding := PaddingLength(n.Size, version)
util.Uint32toBytes(header[0:NeedleChecksumSize], n.Checksum.Value())
- _, err = w.Write(header[0 : NeedleChecksumSize+padding])
- return
+ writeBytes = append(writeBytes, header[0:NeedleChecksumSize+padding]...)
+ return writeBytes, size, actualSize, nil
case Version2, Version3:
header := make([]byte, NeedleHeaderSize+TimestampSize) // adding timestamp to reuse it and avoid extra allocation
CookieToBytes(header[0:CookieSize], n.Cookie)
@@ -92,82 +77,87 @@ func (n *Needle) Append(w *os.File, version Version) (offset uint64, size uint32
} else {
n.Size = 0
}
- size = n.DataSize
util.Uint32toBytes(header[CookieSize+NeedleIdSize:CookieSize+NeedleIdSize+SizeSize], n.Size)
- if _, err = w.Write(header[0:NeedleHeaderSize]); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[0:NeedleHeaderSize]...)
if n.DataSize > 0 {
util.Uint32toBytes(header[0:4], n.DataSize)
- if _, err = w.Write(header[0:4]); err != nil {
- return
- }
- if _, err = w.Write(n.Data); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[0:4]...)
+ writeBytes = append(writeBytes, n.Data...)
util.Uint8toBytes(header[0:1], n.Flags)
- if _, err = w.Write(header[0:1]); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[0:1]...)
if n.HasName() {
util.Uint8toBytes(header[0:1], n.NameSize)
- if _, err = w.Write(header[0:1]); err != nil {
- return
- }
- if _, err = w.Write(n.Name[:n.NameSize]); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[0:1]...)
+ writeBytes = append(writeBytes, n.Name[:n.NameSize]...)
}
if n.HasMime() {
util.Uint8toBytes(header[0:1], n.MimeSize)
- if _, err = w.Write(header[0:1]); err != nil {
- return
- }
- if _, err = w.Write(n.Mime); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[0:1]...)
+ writeBytes = append(writeBytes, n.Mime...)
}
if n.HasLastModifiedDate() {
util.Uint64toBytes(header[0:8], n.LastModified)
- if _, err = w.Write(header[8-LastModifiedBytesLength : 8]); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[8-LastModifiedBytesLength:8]...)
}
if n.HasTtl() && n.Ttl != nil {
n.Ttl.ToBytes(header[0:TtlBytesLength])
- if _, err = w.Write(header[0:TtlBytesLength]); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[0:TtlBytesLength]...)
}
if n.HasPairs() {
util.Uint16toBytes(header[0:2], n.PairsSize)
- if _, err = w.Write(header[0:2]); err != nil {
- return
- }
- if _, err = w.Write(n.Pairs); err != nil {
- return
- }
+ writeBytes = append(writeBytes, header[0:2]...)
+ writeBytes = append(writeBytes, n.Pairs...)
}
}
padding := PaddingLength(n.Size, version)
util.Uint32toBytes(header[0:NeedleChecksumSize], n.Checksum.Value())
if version == Version2 {
- _, err = w.Write(header[0 : NeedleChecksumSize+padding])
+ writeBytes = append(writeBytes, header[0:NeedleChecksumSize+padding]...)
} else {
// version3
util.Uint64toBytes(header[NeedleChecksumSize:NeedleChecksumSize+TimestampSize], n.AppendAtNs)
- _, err = w.Write(header[0 : NeedleChecksumSize+TimestampSize+padding])
+ writeBytes = append(writeBytes, header[0:NeedleChecksumSize+TimestampSize+padding]...)
}
- return offset, n.DataSize, GetActualSize(n.Size, version), err
+ return writeBytes, n.DataSize, GetActualSize(n.Size, version), nil
}
- return 0, 0, 0, fmt.Errorf("Unsupported Version! (%d)", version)
+
+ return writeBytes, 0, 0, fmt.Errorf("Unsupported Version! (%d)", version)
}
-func ReadNeedleBlob(r *os.File, offset int64, size uint32, version Version) (dataSlice []byte, err error) {
- dataSlice = make([]byte, int(GetActualSize(size, version)))
+func (n *Needle) Append(w backend.BackendStorageFile, version Version) (offset uint64, size uint32, actualSize int64, err error) {
+
+ if end, _, e := w.GetStat(); e == nil {
+ defer func(w backend.BackendStorageFile, off int64) {
+ if err != nil {
+ if te := w.Truncate(end); te != nil {
+ glog.V(0).Infof("Failed to truncate %s back to %d with error: %v", w.Name(), end, te)
+ }
+ }
+ }(w, end)
+ offset = uint64(end)
+ } else {
+ err = fmt.Errorf("Cannot Read Current Volume Position: %v", e)
+ return
+ }
+
+ bytesToWrite, size, actualSize, err := n.prepareWriteBuffer(version)
+
+ if err == nil {
+ _, err = w.WriteAt(bytesToWrite, int64(offset))
+ }
+
+ return offset, size, actualSize, err
+}
+
+func ReadNeedleBlob(r backend.BackendStorageFile, offset int64, size uint32, version Version) (dataSlice []byte, err error) {
+
+ dataSize := GetActualSize(size, version)
+ dataSlice = make([]byte, int(dataSize))
+
_, err = r.ReadAt(dataSlice, offset)
return dataSlice, err
+
}
// ReadBytes hydrates the needle from the bytes buffer, with only n.Id is set.
@@ -201,7 +191,7 @@ func (n *Needle) ReadBytes(bytes []byte, offset int64, size uint32, version Vers
}
// ReadData hydrates the needle from the file, with only n.Id is set.
-func (n *Needle) ReadData(r *os.File, offset int64, size uint32, version Version) (err error) {
+func (n *Needle) ReadData(r backend.BackendStorageFile, offset int64, size uint32, version Version) (err error) {
bytes, err := ReadNeedleBlob(r, offset, size, version)
if err != nil {
return err
@@ -276,18 +266,21 @@ func (n *Needle) readNeedleDataVersion2(bytes []byte) (err error) {
return nil
}
-func ReadNeedleHeader(r *os.File, version Version, offset int64) (n *Needle, bytes []byte, bodyLength int64, err error) {
+func ReadNeedleHeader(r backend.BackendStorageFile, version Version, offset int64) (n *Needle, bytes []byte, bodyLength int64, err error) {
n = new(Needle)
if version == Version1 || version == Version2 || version == Version3 {
bytes = make([]byte, NeedleHeaderSize)
+
var count int
count, err = r.ReadAt(bytes, offset)
if count <= 0 || err != nil {
return nil, bytes, 0, err
}
+
n.ParseNeedleHeader(bytes)
bodyLength = NeedleBodyLength(n.Size, version)
}
+
return
}
@@ -308,7 +301,7 @@ func NeedleBodyLength(needleSize uint32, version Version) int64 {
//n should be a needle already read the header
//the input stream will read until next file entry
-func (n *Needle) ReadNeedleBody(r *os.File, version Version, offset int64, bodyLength int64) (bytes []byte, err error) {
+func (n *Needle) ReadNeedleBody(r backend.BackendStorageFile, version Version, offset int64, bodyLength int64) (bytes []byte, err error) {
if bodyLength <= 0 {
return nil, nil
diff --git a/weed/storage/needle/needle_read_write_test.go b/weed/storage/needle/needle_read_write_test.go
index 4c507f9e6..47582dd26 100644
--- a/weed/storage/needle/needle_read_write_test.go
+++ b/weed/storage/needle/needle_read_write_test.go
@@ -5,6 +5,7 @@ import (
"os"
"testing"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -54,7 +55,10 @@ func TestAppend(t *testing.T) {
os.Remove(tempFile.Name())
}()
- offset, _, _, _ := n.Append(tempFile, CurrentVersion)
+ datBackend := backend.NewDiskFile(tempFile)
+ defer datBackend.Close()
+
+ offset, _, _, _ := n.Append(datBackend, CurrentVersion)
if offset != uint64(fileSize) {
t.Errorf("Fail to Append Needle.")
}
diff --git a/weed/storage/needle_map.go b/weed/storage/needle_map.go
index 876839be1..77d081ea7 100644
--- a/weed/storage/needle_map.go
+++ b/weed/storage/needle_map.go
@@ -2,7 +2,6 @@ package storage
import (
"fmt"
- "io/ioutil"
"os"
"sync"
@@ -31,8 +30,6 @@ type NeedleMapper interface {
DeletedCount() int
MaxFileKey() NeedleId
IndexFileSize() uint64
- IndexFileContent() ([]byte, error)
- IndexFileName() string
}
type baseNeedleMapper struct {
@@ -50,10 +47,6 @@ func (nm *baseNeedleMapper) IndexFileSize() uint64 {
return 0
}
-func (nm *baseNeedleMapper) IndexFileName() string {
- return nm.indexFile.Name()
-}
-
func (nm *baseNeedleMapper) appendToIndexFile(key NeedleId, offset Offset, size uint32) error {
bytes := needle_map.ToBytes(key, offset, size)
@@ -66,9 +59,3 @@ func (nm *baseNeedleMapper) appendToIndexFile(key NeedleId, offset Offset, size
_, err := nm.indexFile.Write(bytes)
return err
}
-
-func (nm *baseNeedleMapper) IndexFileContent() ([]byte, error) {
- nm.indexFileAccessLock.Lock()
- defer nm.indexFileAccessLock.Unlock()
- return ioutil.ReadFile(nm.indexFile.Name())
-}
diff --git a/weed/storage/needle_map/btree_map.go b/weed/storage/needle_map/btree_map.go
deleted file mode 100644
index a26c5e068..000000000
--- a/weed/storage/needle_map/btree_map.go
+++ /dev/null
@@ -1,53 +0,0 @@
-package needle_map
-
-import (
- . "github.com/chrislusf/seaweedfs/weed/storage/types"
- "github.com/google/btree"
-)
-
-//This map assumes mostly inserting increasing keys
-type BtreeMap struct {
- tree *btree.BTree
-}
-
-func NewBtreeMap() *BtreeMap {
- return &BtreeMap{
- tree: btree.New(32),
- }
-}
-
-func (cm *BtreeMap) Set(key NeedleId, offset Offset, size uint32) (oldOffset Offset, oldSize uint32) {
- found := cm.tree.ReplaceOrInsert(NeedleValue{key, offset, size})
- if found != nil {
- old := found.(NeedleValue)
- return old.Offset, old.Size
- }
- return
-}
-
-func (cm *BtreeMap) Delete(key NeedleId) (oldSize uint32) {
- found := cm.tree.Delete(NeedleValue{key, Offset{}, 0})
- if found != nil {
- old := found.(NeedleValue)
- return old.Size
- }
- return
-}
-func (cm *BtreeMap) Get(key NeedleId) (*NeedleValue, bool) {
- found := cm.tree.Get(NeedleValue{key, Offset{}, 0})
- if found != nil {
- old := found.(NeedleValue)
- return &old, true
- }
- return nil, false
-}
-
-// Visit visits all entries or stop if any error when visiting
-func (cm *BtreeMap) AscendingVisit(visit func(NeedleValue) error) (ret error) {
- cm.tree.Ascend(func(item btree.Item) bool {
- needle := item.(NeedleValue)
- ret = visit(needle)
- return ret == nil
- })
- return ret
-}
diff --git a/weed/storage/needle_map/compact_map_test.go b/weed/storage/needle_map/compact_map_test.go
index 3bad85727..7eea3969a 100644
--- a/weed/storage/needle_map/compact_map_test.go
+++ b/weed/storage/needle_map/compact_map_test.go
@@ -8,7 +8,14 @@ import (
func TestOverflow2(t *testing.T) {
m := NewCompactMap()
- m.Set(NeedleId(150088), ToOffset(8), 3000073)
+ _, oldSize := m.Set(NeedleId(150088), ToOffset(8), 3000073)
+ if oldSize != 0 {
+ t.Fatalf("expecting no previous data")
+ }
+ _, oldSize = m.Set(NeedleId(150088), ToOffset(8), 3000073)
+ if oldSize != 3000073 {
+ t.Fatalf("expecting previous data size is %d, not %d", 3000073, oldSize)
+ }
m.Set(NeedleId(150073), ToOffset(8), 3000073)
m.Set(NeedleId(150089), ToOffset(8), 3000073)
m.Set(NeedleId(150076), ToOffset(8), 3000073)
diff --git a/weed/storage/needle_map/memdb.go b/weed/storage/needle_map/memdb.go
new file mode 100644
index 000000000..6aba6adeb
--- /dev/null
+++ b/weed/storage/needle_map/memdb.go
@@ -0,0 +1,112 @@
+package needle_map
+
+import (
+ "fmt"
+ "os"
+
+ "github.com/syndtr/goleveldb/leveldb"
+ "github.com/syndtr/goleveldb/leveldb/opt"
+ "github.com/syndtr/goleveldb/leveldb/storage"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/idx"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+//This map uses in memory level db
+type MemDb struct {
+ db *leveldb.DB
+}
+
+func NewMemDb() *MemDb {
+ opts := &opt.Options{}
+
+ var err error
+ t := &MemDb{}
+ if t.db, err = leveldb.Open(storage.NewMemStorage(), opts); err != nil {
+ glog.V(0).Infof("MemDb fails to open: %v", err)
+ return nil
+ }
+
+ return t
+}
+
+func (cm *MemDb) Set(key NeedleId, offset Offset, size uint32) error {
+
+ bytes := ToBytes(key, offset, size)
+
+ if err := cm.db.Put(bytes[0:NeedleIdSize], bytes[NeedleIdSize:NeedleIdSize+OffsetSize+SizeSize], nil); err != nil {
+ return fmt.Errorf("failed to write temp leveldb: %v", err)
+ }
+ return nil
+}
+
+func (cm *MemDb) Delete(key NeedleId) error {
+ bytes := make([]byte, NeedleIdSize)
+ NeedleIdToBytes(bytes, key)
+ return cm.db.Delete(bytes, nil)
+
+}
+func (cm *MemDb) Get(key NeedleId) (*NeedleValue, bool) {
+ bytes := make([]byte, NeedleIdSize)
+ NeedleIdToBytes(bytes[0:NeedleIdSize], key)
+ data, err := cm.db.Get(bytes, nil)
+ if err != nil || len(data) != OffsetSize+SizeSize {
+ return nil, false
+ }
+ offset := BytesToOffset(data[0:OffsetSize])
+ size := util.BytesToUint32(data[OffsetSize : OffsetSize+SizeSize])
+ return &NeedleValue{Key: key, Offset: offset, Size: size}, true
+}
+
+// Visit visits all entries or stop if any error when visiting
+func (cm *MemDb) AscendingVisit(visit func(NeedleValue) error) (ret error) {
+ iter := cm.db.NewIterator(nil, nil)
+ for iter.Next() {
+ key := BytesToNeedleId(iter.Key())
+ data := iter.Value()
+ offset := BytesToOffset(data[0:OffsetSize])
+ size := util.BytesToUint32(data[OffsetSize : OffsetSize+SizeSize])
+
+ needle := NeedleValue{Key: key, Offset: offset, Size: size}
+ ret = visit(needle)
+ if ret != nil {
+ return
+ }
+ }
+ iter.Release()
+ ret = iter.Error()
+
+ return
+}
+
+func (cm *MemDb) SaveToIdx(idxName string) (ret error) {
+ idxFile, err := os.OpenFile(idxName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if err != nil {
+ return
+ }
+ defer idxFile.Close()
+
+ return cm.AscendingVisit(func(value NeedleValue) error {
+ _, err := idxFile.Write(value.ToBytes())
+ return err
+ })
+
+}
+
+func (cm *MemDb) LoadFromIdx(idxName string) (ret error) {
+ idxFile, err := os.OpenFile(idxName, os.O_RDONLY, 0644)
+ if err != nil {
+ return
+ }
+ defer idxFile.Close()
+
+ return idx.WalkIndexFile(idxFile, func(key NeedleId, offset Offset, size uint32) error {
+ if offset.IsZero() || size == TombstoneFileSize {
+ return nil
+ }
+ return cm.Set(key, offset, size)
+ })
+
+}
diff --git a/weed/storage/needle_map_leveldb.go b/weed/storage/needle_map_leveldb.go
index aa726e733..ef8571e83 100644
--- a/weed/storage/needle_map_leveldb.go
+++ b/weed/storage/needle_map_leveldb.go
@@ -101,13 +101,9 @@ func (m *LevelDbNeedleMap) Put(key NeedleId, offset Offset, size uint32) error {
return levelDbWrite(m.db, key, offset, size)
}
-func levelDbWrite(db *leveldb.DB,
- key NeedleId, offset Offset, size uint32) error {
+func levelDbWrite(db *leveldb.DB, key NeedleId, offset Offset, size uint32) error {
- bytes := make([]byte, NeedleIdSize+OffsetSize+SizeSize)
- NeedleIdToBytes(bytes[0:NeedleIdSize], key)
- OffsetToBytes(bytes[NeedleIdSize:NeedleIdSize+OffsetSize], offset)
- util.Uint32toBytes(bytes[NeedleIdSize+OffsetSize:NeedleIdSize+OffsetSize+SizeSize], size)
+ bytes := needle_map.ToBytes(key, offset, size)
if err := db.Put(bytes[0:NeedleIdSize], bytes[NeedleIdSize:NeedleIdSize+OffsetSize+SizeSize], nil); err != nil {
return fmt.Errorf("failed to write leveldb: %v", err)
diff --git a/weed/storage/needle_map_memory.go b/weed/storage/needle_map_memory.go
index ee639a7e6..37dee7889 100644
--- a/weed/storage/needle_map_memory.go
+++ b/weed/storage/needle_map_memory.go
@@ -22,24 +22,11 @@ func NewCompactNeedleMap(file *os.File) *NeedleMap {
return nm
}
-func NewBtreeNeedleMap(file *os.File) *NeedleMap {
- nm := &NeedleMap{
- m: needle_map.NewBtreeMap(),
- }
- nm.indexFile = file
- return nm
-}
-
func LoadCompactNeedleMap(file *os.File) (*NeedleMap, error) {
nm := NewCompactNeedleMap(file)
return doLoading(file, nm)
}
-func LoadBtreeNeedleMap(file *os.File) (*NeedleMap, error) {
- nm := NewBtreeNeedleMap(file)
- return doLoading(file, nm)
-}
-
func doLoading(file *os.File, nm *NeedleMap) (*NeedleMap, error) {
e := idx.WalkIndexFile(file, func(key NeedleId, offset Offset, size uint32) error {
nm.MaybeSetMaxFileKey(key)
@@ -47,14 +34,12 @@ func doLoading(file *os.File, nm *NeedleMap) (*NeedleMap, error) {
nm.FileCounter++
nm.FileByteCounter = nm.FileByteCounter + uint64(size)
oldOffset, oldSize := nm.m.Set(NeedleId(key), offset, size)
- // glog.V(3).Infoln("reading key", key, "offset", offset*NeedlePaddingSize, "size", size, "oldSize", oldSize)
if !oldOffset.IsZero() && oldSize != TombstoneFileSize {
nm.DeletionCounter++
nm.DeletionByteCounter = nm.DeletionByteCounter + uint64(oldSize)
}
} else {
oldSize := nm.m.Delete(NeedleId(key))
- // glog.V(3).Infoln("removing key", key, "offset", offset*NeedlePaddingSize, "size", size, "oldSize", oldSize)
nm.DeletionCounter++
nm.DeletionByteCounter = nm.DeletionByteCounter + uint64(oldSize)
}
diff --git a/weed/storage/needle_map_metric.go b/weed/storage/needle_map_metric.go
index 6448b053b..823a04108 100644
--- a/weed/storage/needle_map_metric.go
+++ b/weed/storage/needle_map_metric.go
@@ -19,10 +19,16 @@ type mapMetric struct {
}
func (mm *mapMetric) logDelete(deletedByteCount uint32) {
+ if mm == nil {
+ return
+ }
mm.LogDeletionCounter(deletedByteCount)
}
func (mm *mapMetric) logPut(key NeedleId, oldSize uint32, newSize uint32) {
+ if mm == nil {
+ return
+ }
mm.MaybeSetMaxFileKey(key)
mm.LogFileCounter(newSize)
if oldSize > 0 && oldSize != TombstoneFileSize {
@@ -30,32 +36,56 @@ func (mm *mapMetric) logPut(key NeedleId, oldSize uint32, newSize uint32) {
}
}
func (mm *mapMetric) LogFileCounter(newSize uint32) {
+ if mm == nil {
+ return
+ }
atomic.AddUint32(&mm.FileCounter, 1)
atomic.AddUint64(&mm.FileByteCounter, uint64(newSize))
}
func (mm *mapMetric) LogDeletionCounter(oldSize uint32) {
+ if mm == nil {
+ return
+ }
if oldSize > 0 {
atomic.AddUint32(&mm.DeletionCounter, 1)
atomic.AddUint64(&mm.DeletionByteCounter, uint64(oldSize))
}
}
func (mm *mapMetric) ContentSize() uint64 {
+ if mm == nil {
+ return 0
+ }
return atomic.LoadUint64(&mm.FileByteCounter)
}
func (mm *mapMetric) DeletedSize() uint64 {
+ if mm == nil {
+ return 0
+ }
return atomic.LoadUint64(&mm.DeletionByteCounter)
}
func (mm *mapMetric) FileCount() int {
+ if mm == nil {
+ return 0
+ }
return int(atomic.LoadUint32(&mm.FileCounter))
}
func (mm *mapMetric) DeletedCount() int {
+ if mm == nil {
+ return 0
+ }
return int(atomic.LoadUint32(&mm.DeletionCounter))
}
func (mm *mapMetric) MaxFileKey() NeedleId {
+ if mm == nil {
+ return 0
+ }
t := uint64(mm.MaximumFileKey)
return Uint64ToNeedleId(t)
}
func (mm *mapMetric) MaybeSetMaxFileKey(key NeedleId) {
+ if mm == nil {
+ return
+ }
if key > mm.MaxFileKey() {
atomic.StoreUint64(&mm.MaximumFileKey, uint64(key))
}
diff --git a/weed/storage/needle_map_metric_test.go b/weed/storage/needle_map_metric_test.go
index 539f83a87..ae2177a30 100644
--- a/weed/storage/needle_map_metric_test.go
+++ b/weed/storage/needle_map_metric_test.go
@@ -1,17 +1,18 @@
package storage
import (
- "github.com/chrislusf/seaweedfs/weed/glog"
- . "github.com/chrislusf/seaweedfs/weed/storage/types"
"io/ioutil"
"math/rand"
"testing"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
)
func TestFastLoadingNeedleMapMetrics(t *testing.T) {
idxFile, _ := ioutil.TempFile("", "tmp.idx")
- nm := NewBtreeNeedleMap(idxFile)
+ nm := NewCompactNeedleMap(idxFile)
for i := 0; i < 10000; i++ {
nm.Put(Uint64ToNeedleId(uint64(i+1)), Uint32ToOffset(uint32(0)), uint32(1))
diff --git a/weed/storage/needle_map_sorted_file.go b/weed/storage/needle_map_sorted_file.go
new file mode 100644
index 000000000..e6f9258f3
--- /dev/null
+++ b/weed/storage/needle_map_sorted_file.go
@@ -0,0 +1,105 @@
+package storage
+
+import (
+ "os"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+type SortedFileNeedleMap struct {
+ baseNeedleMapper
+ baseFileName string
+ dbFile *os.File
+ dbFileSize int64
+}
+
+func NewSortedFileNeedleMap(baseFileName string, indexFile *os.File) (m *SortedFileNeedleMap, err error) {
+ m = &SortedFileNeedleMap{baseFileName: baseFileName}
+ m.indexFile = indexFile
+ fileName := baseFileName + ".sdx"
+ if !isSortedFileFresh(fileName, indexFile) {
+ glog.V(0).Infof("Start to Generate %s from %s", fileName, indexFile.Name())
+ erasure_coding.WriteSortedFileFromIdx(baseFileName, ".sdx")
+ glog.V(0).Infof("Finished Generating %s from %s", fileName, indexFile.Name())
+ }
+ glog.V(1).Infof("Opening %s...", fileName)
+
+ if m.dbFile, err = os.Open(baseFileName + ".sdx"); err != nil {
+ return
+ }
+ dbStat, _ := m.dbFile.Stat()
+ m.dbFileSize = dbStat.Size()
+ glog.V(1).Infof("Loading %s...", indexFile.Name())
+ mm, indexLoadError := newNeedleMapMetricFromIndexFile(indexFile)
+ if indexLoadError != nil {
+ return nil, indexLoadError
+ }
+ m.mapMetric = *mm
+ return
+}
+
+func isSortedFileFresh(dbFileName string, indexFile *os.File) bool {
+ // normally we always write to index file first
+ dbFile, err := os.Open(dbFileName)
+ if err != nil {
+ return false
+ }
+ defer dbFile.Close()
+ dbStat, dbStatErr := dbFile.Stat()
+ indexStat, indexStatErr := indexFile.Stat()
+ if dbStatErr != nil || indexStatErr != nil {
+ glog.V(0).Infof("Can not stat file: %v and %v", dbStatErr, indexStatErr)
+ return false
+ }
+
+ return dbStat.ModTime().After(indexStat.ModTime())
+}
+
+func (m *SortedFileNeedleMap) Get(key NeedleId) (element *needle_map.NeedleValue, ok bool) {
+ offset, size, err := erasure_coding.SearchNeedleFromSortedIndex(m.dbFile, m.dbFileSize, key, nil)
+ ok = err == nil
+ return &needle_map.NeedleValue{Key: key, Offset: offset, Size: size}, ok
+
+}
+
+func (m *SortedFileNeedleMap) Put(key NeedleId, offset Offset, size uint32) error {
+ return os.ErrInvalid
+}
+
+func (m *SortedFileNeedleMap) Delete(key NeedleId, offset Offset) error {
+
+ _, size, err := erasure_coding.SearchNeedleFromSortedIndex(m.dbFile, m.dbFileSize, key, nil)
+
+ if err != nil {
+ if err == erasure_coding.NotFoundError {
+ return nil
+ }
+ return err
+ }
+
+ if size == TombstoneFileSize {
+ return nil
+ }
+
+ // write to index file first
+ if err := m.appendToIndexFile(key, offset, TombstoneFileSize); err != nil {
+ return err
+ }
+ _, _, err = erasure_coding.SearchNeedleFromSortedIndex(m.dbFile, m.dbFileSize, key, erasure_coding.MarkNeedleDeleted)
+
+ return err
+}
+
+func (m *SortedFileNeedleMap) Close() {
+ m.indexFile.Close()
+ m.dbFile.Close()
+}
+
+func (m *SortedFileNeedleMap) Destroy() error {
+ m.Close()
+ os.Remove(m.indexFile.Name())
+ return os.Remove(m.baseFileName + ".sdx")
+}
diff --git a/weed/storage/store.go b/weed/storage/store.go
index ac13f6a28..512f72ceb 100644
--- a/weed/storage/store.go
+++ b/weed/storage/store.go
@@ -4,12 +4,14 @@ import (
"fmt"
"sync/atomic"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/stats"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
- "google.golang.org/grpc"
)
const (
@@ -59,8 +61,8 @@ func NewStore(grpcDialOption grpc.DialOption, port int, ip, publicUrl string, di
return
}
-func (s *Store) AddVolume(volumeId needle.VolumeId, collection string, needleMapKind NeedleMapType, replicaPlacement string, ttlString string, preallocate int64) error {
- rt, e := NewReplicaPlacementFromString(replicaPlacement)
+func (s *Store) AddVolume(volumeId needle.VolumeId, collection string, needleMapKind NeedleMapType, replicaPlacement string, ttlString string, preallocate int64, MemoryMapMaxSizeMb uint32) error {
+ rt, e := super_block.NewReplicaPlacementFromString(replicaPlacement)
if e != nil {
return e
}
@@ -68,7 +70,7 @@ func (s *Store) AddVolume(volumeId needle.VolumeId, collection string, needleMap
if e != nil {
return e
}
- e = s.addVolume(volumeId, collection, needleMapKind, rt, ttl, preallocate)
+ e = s.addVolume(volumeId, collection, needleMapKind, rt, ttl, preallocate, MemoryMapMaxSizeMb)
return e
}
func (s *Store) DeleteCollection(collection string) (e error) {
@@ -101,14 +103,14 @@ func (s *Store) FindFreeLocation() (ret *DiskLocation) {
}
return ret
}
-func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind NeedleMapType, replicaPlacement *ReplicaPlacement, ttl *needle.TTL, preallocate int64) error {
+func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind NeedleMapType, replicaPlacement *super_block.ReplicaPlacement, ttl *needle.TTL, preallocate int64, memoryMapMaxSizeMb uint32) error {
if s.findVolume(vid) != nil {
return fmt.Errorf("Volume Id %d already exists!", vid)
}
if location := s.FindFreeLocation(); location != nil {
glog.V(0).Infof("In dir %s adds volume:%v collection:%s replicaPlacement:%v ttl:%v",
location.Directory, vid, collection, replicaPlacement, ttl)
- if volume, err := NewVolume(location.Directory, collection, vid, needleMapKind, replicaPlacement, ttl, preallocate); err == nil {
+ if volume, err := NewVolume(location.Directory, collection, vid, needleMapKind, replicaPlacement, ttl, preallocate, memoryMapMaxSizeMb); err == nil {
location.SetVolume(vid, volume)
glog.V(0).Infof("add volume %d", vid)
s.NewVolumesChan <- master_pb.VolumeShortInformationMessage{
@@ -126,10 +128,10 @@ func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind
return fmt.Errorf("No more free space left")
}
-func (s *Store) Status() []*VolumeInfo {
+func (s *Store) VolumeInfos() []*VolumeInfo {
var stats []*VolumeInfo
for _, location := range s.Locations {
- location.RLock()
+ location.volumesLock.RLock()
for k, v := range location.volumes {
s := &VolumeInfo{
Id: needle.VolumeId(k),
@@ -137,16 +139,17 @@ func (s *Store) Status() []*VolumeInfo {
Collection: v.Collection,
ReplicaPlacement: v.ReplicaPlacement,
Version: v.Version(),
- FileCount: v.nm.FileCount(),
- DeleteCount: v.nm.DeletedCount(),
- DeletedByteCount: v.nm.DeletedSize(),
- ReadOnly: v.readOnly,
+ FileCount: int(v.FileCount()),
+ DeleteCount: int(v.DeletedCount()),
+ DeletedByteCount: v.DeletedSize(),
+ ReadOnly: v.noWriteOrDelete || v.noWriteCanDelete,
Ttl: v.Ttl,
CompactRevision: uint32(v.CompactionRevision),
}
+ s.RemoteStorageName, s.RemoteStorageKey = v.RemoteStorageNameKey()
stats = append(stats, s)
}
- location.RUnlock()
+ location.volumesLock.RUnlock()
}
sortVolumeInfos(stats)
return stats
@@ -165,18 +168,18 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
var maxFileKey NeedleId
collectionVolumeSize := make(map[string]uint64)
for _, location := range s.Locations {
+ var deleteVids []needle.VolumeId
maxVolumeCount = maxVolumeCount + location.MaxVolumeCount
- location.Lock()
+ location.volumesLock.RLock()
for _, v := range location.volumes {
- if maxFileKey < v.nm.MaxFileKey() {
- maxFileKey = v.nm.MaxFileKey()
+ if maxFileKey < v.MaxFileKey() {
+ maxFileKey = v.MaxFileKey()
}
if !v.expired(s.GetVolumeSizeLimit()) {
volumeMessages = append(volumeMessages, v.ToVolumeInformationMessage())
} else {
if v.expiredLongEnough(MAX_TTL_VOLUME_REMOVAL_DELAY) {
- location.deleteVolumeById(v.Id)
- glog.V(0).Infoln("volume", v.Id, "is deleted.")
+ deleteVids = append(deleteVids, v.Id)
} else {
glog.V(0).Infoln("volume", v.Id, "is expired.")
}
@@ -184,7 +187,17 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
fileSize, _, _ := v.FileStat()
collectionVolumeSize[v.Collection] += fileSize
}
- location.Unlock()
+ location.volumesLock.RUnlock()
+
+ if len(deleteVids) > 0 {
+ // delete expired volumes.
+ location.volumesLock.Lock()
+ for _, vid := range deleteVids {
+ location.deleteVolumeById(vid)
+ glog.V(0).Infoln("volume", vid, "is deleted.")
+ }
+ location.volumesLock.Unlock()
+ }
}
for col, size := range collectionVolumeSize {
@@ -213,11 +226,11 @@ func (s *Store) Close() {
func (s *Store) WriteVolumeNeedle(i needle.VolumeId, n *needle.Needle) (size uint32, isUnchanged bool, err error) {
if v := s.findVolume(i); v != nil {
- if v.readOnly {
+ if v.noWriteOrDelete || v.noWriteCanDelete {
err = fmt.Errorf("volume %d is read only", i)
return
}
- if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(size, v.version)) {
+ if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(size, v.Version())) {
_, size, isUnchanged, err = v.writeNeedle(n)
} else {
err = fmt.Errorf("volume size limit %d exceeded! current size is %d", s.GetVolumeSizeLimit(), v.ContentSize())
@@ -231,10 +244,10 @@ func (s *Store) WriteVolumeNeedle(i needle.VolumeId, n *needle.Needle) (size uin
func (s *Store) DeleteVolumeNeedle(i needle.VolumeId, n *needle.Needle) (uint32, error) {
if v := s.findVolume(i); v != nil {
- if v.readOnly {
+ if v.noWriteOrDelete {
return 0, fmt.Errorf("volume %d is read only", i)
}
- if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(0, v.version)) {
+ if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(0, v.Version())) {
return v.deleteNeedle(n)
} else {
return 0, fmt.Errorf("volume size limit %d exceeded! current size is %d", s.GetVolumeSizeLimit(), v.ContentSize())
@@ -263,7 +276,7 @@ func (s *Store) MarkVolumeReadonly(i needle.VolumeId) error {
if v == nil {
return fmt.Errorf("volume %d not found", i)
}
- v.readOnly = true
+ v.noWriteOrDelete = true
return nil
}
diff --git a/weed/storage/store_ec.go b/weed/storage/store_ec.go
index 8271324cf..27406451f 100644
--- a/weed/storage/store_ec.go
+++ b/weed/storage/store_ec.go
@@ -8,6 +8,8 @@ import (
"sync"
"time"
+ "github.com/klauspost/reedsolomon"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
@@ -16,7 +18,6 @@ import (
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/types"
- "github.com/klauspost/reedsolomon"
)
func (s *Store) CollectErasureCodingHeartbeat() *master_pb.Heartbeat {
@@ -119,15 +120,7 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
for _, location := range s.Locations {
if localEcVolume, found := location.FindEcVolume(vid); found {
- // read the volume version
- for localEcVolume.Version == 0 {
- err := s.readEcVolumeVersion(ctx, vid, localEcVolume)
- time.Sleep(1357 * time.Millisecond)
- glog.V(0).Infof("ReadEcShardNeedle vid %d version:%v: %v", vid, localEcVolume.Version, err)
- }
- version := localEcVolume.Version
-
- offset, size, intervals, err := localEcVolume.LocateEcShardNeedle(n.Id, version)
+ offset, size, intervals, err := localEcVolume.LocateEcShardNeedle(n.Id, localEcVolume.Version)
if err != nil {
return 0, fmt.Errorf("locate in local ec volume: %v", err)
}
@@ -148,7 +141,7 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
return 0, fmt.Errorf("ec entry %s is deleted", n.Id)
}
- err = n.ReadBytes(bytes, offset.ToAcutalOffset(), size, version)
+ err = n.ReadBytes(bytes, offset.ToAcutalOffset(), size, localEcVolume.Version)
if err != nil {
return 0, fmt.Errorf("readbytes: %v", err)
}
@@ -159,22 +152,6 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
return 0, fmt.Errorf("ec shard %d not found", vid)
}
-func (s *Store) readEcVolumeVersion(ctx context.Context, vid needle.VolumeId, ecVolume *erasure_coding.EcVolume) (err error) {
-
- interval := erasure_coding.Interval{
- BlockIndex: 0,
- InnerBlockOffset: 0,
- Size: _SuperBlockSize,
- IsLargeBlock: true, // it could be large block, but ok in this place
- LargeBlockRowsCount: 0,
- }
- data, _, err := s.readEcShardIntervals(ctx, vid, 0, ecVolume, []erasure_coding.Interval{interval})
- if err == nil {
- ecVolume.Version = needle.Version(data[0])
- }
- return
-}
-
func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
if err = s.cachedLookupEcShardLocations(ctx, ecVolume); err != nil {
@@ -288,7 +265,7 @@ func (s *Store) readRemoteEcShardInterval(ctx context.Context, sourceDataNodes [
}
for _, sourceDataNode := range sourceDataNodes {
- glog.V(4).Infof("read remote ec shard %d.%d from %s", vid, shardId, sourceDataNode)
+ glog.V(3).Infof("read remote ec shard %d.%d from %s", vid, shardId, sourceDataNode)
n, is_deleted, err = s.doReadRemoteEcShardInterval(ctx, sourceDataNode, needleId, vid, shardId, buf, offset)
if err == nil {
return
@@ -340,7 +317,7 @@ func (s *Store) doReadRemoteEcShardInterval(ctx context.Context, sourceDataNode
}
func (s *Store) recoverOneRemoteEcShardInterval(ctx context.Context, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, shardIdToRecover erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
- glog.V(4).Infof("recover ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
+ glog.V(3).Infof("recover ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
enc, err := reedsolomon.New(erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount)
if err != nil {
diff --git a/weed/storage/store_vacuum.go b/weed/storage/store_vacuum.go
index b1f1a6277..5dacb71bf 100644
--- a/weed/storage/store_vacuum.go
+++ b/weed/storage/store_vacuum.go
@@ -16,7 +16,7 @@ func (s *Store) CheckCompactVolume(volumeId needle.VolumeId) (float64, error) {
}
func (s *Store) CompactVolume(vid needle.VolumeId, preallocate int64, compactionBytePerSecond int64) error {
if v := s.findVolume(vid); v != nil {
- return v.Compact(preallocate, compactionBytePerSecond)
+ return v.Compact2(preallocate) // compactionBytePerSecond
}
return fmt.Errorf("volume id %d is not found during compact", vid)
}
diff --git a/weed/storage/replica_placement.go b/weed/storage/super_block/replica_placement.go
similarity index 98%
rename from weed/storage/replica_placement.go
rename to weed/storage/super_block/replica_placement.go
index c1aca52eb..fcccbba7d 100644
--- a/weed/storage/replica_placement.go
+++ b/weed/storage/super_block/replica_placement.go
@@ -1,4 +1,4 @@
-package storage
+package super_block
import (
"errors"
diff --git a/weed/storage/replica_placement_test.go b/weed/storage/super_block/replica_placement_test.go
similarity index 93%
rename from weed/storage/replica_placement_test.go
rename to weed/storage/super_block/replica_placement_test.go
index 7968af7cb..7742ba548 100644
--- a/weed/storage/replica_placement_test.go
+++ b/weed/storage/super_block/replica_placement_test.go
@@ -1,4 +1,4 @@
-package storage
+package super_block
import (
"testing"
diff --git a/weed/storage/super_block/super_block.go b/weed/storage/super_block/super_block.go
new file mode 100644
index 000000000..f48cd0bdc
--- /dev/null
+++ b/weed/storage/super_block/super_block.go
@@ -0,0 +1,69 @@
+package super_block
+
+import (
+ "github.com/golang/protobuf/proto"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+const (
+ SuperBlockSize = 8
+)
+
+/*
+* Super block currently has 8 bytes allocated for each volume.
+* Byte 0: version, 1 or 2
+* Byte 1: Replica Placement strategy, 000, 001, 002, 010, etc
+* Byte 2 and byte 3: Time to live. See TTL for definition
+* Byte 4 and byte 5: The number of times the volume has been compacted.
+* Rest bytes: Reserved
+ */
+type SuperBlock struct {
+ Version needle.Version
+ ReplicaPlacement *ReplicaPlacement
+ Ttl *needle.TTL
+ CompactionRevision uint16
+ Extra *master_pb.SuperBlockExtra
+ ExtraSize uint16
+}
+
+func (s *SuperBlock) BlockSize() int {
+ switch s.Version {
+ case needle.Version2, needle.Version3:
+ return SuperBlockSize + int(s.ExtraSize)
+ }
+ return SuperBlockSize
+}
+
+func (s *SuperBlock) Bytes() []byte {
+ header := make([]byte, SuperBlockSize)
+ header[0] = byte(s.Version)
+ header[1] = s.ReplicaPlacement.Byte()
+ s.Ttl.ToBytes(header[2:4])
+ util.Uint16toBytes(header[4:6], s.CompactionRevision)
+
+ if s.Extra != nil {
+ extraData, err := proto.Marshal(s.Extra)
+ if err != nil {
+ glog.Fatalf("cannot marshal super block extra %+v: %v", s.Extra, err)
+ }
+ extraSize := len(extraData)
+ if extraSize > 256*256-2 {
+ // reserve a couple of bits for future extension
+ glog.Fatalf("super block extra size is %d bigger than %d", extraSize, 256*256-2)
+ }
+ s.ExtraSize = uint16(extraSize)
+ util.Uint16toBytes(header[6:8], s.ExtraSize)
+
+ header = append(header, extraData...)
+ }
+
+ return header
+}
+
+func (s *SuperBlock) Initialized() bool {
+ return s.ReplicaPlacement != nil && s.Ttl != nil
+}
diff --git a/weed/storage/super_block/super_block_read.go.go b/weed/storage/super_block/super_block_read.go.go
new file mode 100644
index 000000000..9eb12e116
--- /dev/null
+++ b/weed/storage/super_block/super_block_read.go.go
@@ -0,0 +1,44 @@
+package super_block
+
+import (
+ "fmt"
+
+ "github.com/golang/protobuf/proto"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+// ReadSuperBlock reads from data file and load it into volume's super block
+func ReadSuperBlock(datBackend backend.BackendStorageFile) (superBlock SuperBlock, err error) {
+
+ header := make([]byte, SuperBlockSize)
+ if _, e := datBackend.ReadAt(header, 0); e != nil {
+ err = fmt.Errorf("cannot read volume %s super block: %v", datBackend.Name(), e)
+ return
+ }
+
+ superBlock.Version = needle.Version(header[0])
+ if superBlock.ReplicaPlacement, err = NewReplicaPlacementFromByte(header[1]); err != nil {
+ err = fmt.Errorf("cannot read replica type: %s", err.Error())
+ return
+ }
+ superBlock.Ttl = needle.LoadTTLFromBytes(header[2:4])
+ superBlock.CompactionRevision = util.BytesToUint16(header[4:6])
+ superBlock.ExtraSize = util.BytesToUint16(header[6:8])
+
+ if superBlock.ExtraSize > 0 {
+ // read more
+ extraData := make([]byte, int(superBlock.ExtraSize))
+ superBlock.Extra = &master_pb.SuperBlockExtra{}
+ err = proto.Unmarshal(extraData, superBlock.Extra)
+ if err != nil {
+ err = fmt.Errorf("cannot read volume %s super block extra: %v", datBackend.Name(), err)
+ return
+ }
+ }
+
+ return
+}
diff --git a/weed/storage/volume_super_block_test.go b/weed/storage/super_block/super_block_test.go
similarity index 86%
rename from weed/storage/volume_super_block_test.go
rename to weed/storage/super_block/super_block_test.go
index 06ad8a5d3..25699070d 100644
--- a/weed/storage/volume_super_block_test.go
+++ b/weed/storage/super_block/super_block_test.go
@@ -1,4 +1,4 @@
-package storage
+package super_block
import (
"testing"
@@ -10,7 +10,7 @@ func TestSuperBlockReadWrite(t *testing.T) {
rp, _ := NewReplicaPlacementFromByte(byte(001))
ttl, _ := needle.ReadTTL("15d")
s := &SuperBlock{
- version: needle.CurrentVersion,
+ Version: needle.CurrentVersion,
ReplicaPlacement: rp,
Ttl: ttl,
}
diff --git a/weed/storage/volume.go b/weed/storage/volume.go
index a1d9d7e8d..acede66bf 100644
--- a/weed/storage/volume.go
+++ b/weed/storage/volume.go
@@ -2,50 +2,58 @@ package storage
import (
"fmt"
-
- "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
- "github.com/chrislusf/seaweedfs/weed/stats"
- "github.com/chrislusf/seaweedfs/weed/storage/needle"
-
- "os"
"path"
"strconv"
"sync"
"time"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+ "github.com/chrislusf/seaweedfs/weed/storage/types"
+
"github.com/chrislusf/seaweedfs/weed/glog"
)
type Volume struct {
- Id needle.VolumeId
- dir string
- Collection string
- dataFile *os.File
- nm NeedleMapper
- compactingWg sync.WaitGroup
- needleMapKind NeedleMapType
- readOnly bool
-
- SuperBlock
-
- dataFileAccessLock sync.Mutex
+ Id needle.VolumeId
+ dir string
+ Collection string
+ DataBackend backend.BackendStorageFile
+ nm NeedleMapper
+ needleMapKind NeedleMapType
+ noWriteOrDelete bool // if readonly, either noWriteOrDelete or noWriteCanDelete
+ noWriteCanDelete bool // if readonly, either noWriteOrDelete or noWriteCanDelete
+ hasRemoteFile bool // if the volume has a remote file
+ MemoryMapMaxSizeMb uint32
+
+ super_block.SuperBlock
+
+ dataFileAccessLock sync.RWMutex
lastModifiedTsSeconds uint64 //unix time in seconds
lastAppendAtNs uint64 //unix time in nanoseconds
lastCompactIndexOffset uint64
lastCompactRevision uint16
+
+ isCompacting bool
+
+ volumeInfo *volume_server_pb.VolumeInfo
}
-func NewVolume(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType, replicaPlacement *ReplicaPlacement, ttl *needle.TTL, preallocate int64) (v *Volume, e error) {
+func NewVolume(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType, replicaPlacement *super_block.ReplicaPlacement, ttl *needle.TTL, preallocate int64, memoryMapMaxSizeMb uint32) (v *Volume, e error) {
// if replicaPlacement is nil, the superblock will be loaded from disk
- v = &Volume{dir: dirname, Collection: collection, Id: id}
- v.SuperBlock = SuperBlock{ReplicaPlacement: replicaPlacement, Ttl: ttl}
+ v = &Volume{dir: dirname, Collection: collection, Id: id, MemoryMapMaxSizeMb: memoryMapMaxSizeMb}
+ v.SuperBlock = super_block.SuperBlock{ReplicaPlacement: replicaPlacement, Ttl: ttl}
v.needleMapKind = needleMapKind
e = v.load(true, true, needleMapKind, preallocate)
return
}
func (v *Volume) String() string {
- return fmt.Sprintf("Id:%v, dir:%s, Collection:%s, dataFile:%v, nm:%v, readOnly:%v", v.Id, v.dir, v.Collection, v.dataFile, v.nm, v.readOnly)
+ return fmt.Sprintf("Id:%v, dir:%s, Collection:%s, dataFile:%v, nm:%v, noWrite:%v canDelete:%v", v.Id, v.dir, v.Collection, v.DataBackend, v.nm, v.noWriteOrDelete || v.noWriteCanDelete, v.noWriteCanDelete)
}
func VolumeFileName(dir string, collection string, id int) (fileName string) {
@@ -60,38 +68,84 @@ func VolumeFileName(dir string, collection string, id int) (fileName string) {
func (v *Volume) FileName() (fileName string) {
return VolumeFileName(v.dir, v.Collection, int(v.Id))
}
-func (v *Volume) DataFile() *os.File {
- return v.dataFile
-}
func (v *Volume) Version() needle.Version {
- return v.SuperBlock.Version()
+ if v.volumeInfo.Version != 0 {
+ v.SuperBlock.Version = needle.Version(v.volumeInfo.Version)
+ }
+ return v.SuperBlock.Version
}
func (v *Volume) FileStat() (datSize uint64, idxSize uint64, modTime time.Time) {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
- if v.dataFile == nil {
+ if v.DataBackend == nil {
return
}
- stat, e := v.dataFile.Stat()
+ datFileSize, modTime, e := v.DataBackend.GetStat()
if e == nil {
- return uint64(stat.Size()), v.nm.IndexFileSize(), stat.ModTime()
+ return uint64(datFileSize), v.nm.IndexFileSize(), modTime
}
- glog.V(0).Infof("Failed to read file size %s %v", v.dataFile.Name(), e)
+ glog.V(0).Infof("Failed to read file size %s %v", v.DataBackend.Name(), e)
return // -1 causes integer overflow and the volume to become unwritable.
}
-func (v *Volume) IndexFileSize() uint64 {
- return v.nm.IndexFileSize()
+func (v *Volume) ContentSize() uint64 {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+ if v.nm == nil {
+ return 0
+ }
+ return v.nm.ContentSize()
+}
+
+func (v *Volume) DeletedSize() uint64 {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+ if v.nm == nil {
+ return 0
+ }
+ return v.nm.DeletedSize()
}
func (v *Volume) FileCount() uint64 {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+ if v.nm == nil {
+ return 0
+ }
return uint64(v.nm.FileCount())
}
+func (v *Volume) DeletedCount() uint64 {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+ if v.nm == nil {
+ return 0
+ }
+ return uint64(v.nm.DeletedCount())
+}
+
+func (v *Volume) MaxFileKey() types.NeedleId {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+ if v.nm == nil {
+ return 0
+ }
+ return v.nm.MaxFileKey()
+}
+
+func (v *Volume) IndexFileSize() uint64 {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+ if v.nm == nil {
+ return 0
+ }
+ return v.nm.IndexFileSize()
+}
+
// Close cleanly shuts down this volume
func (v *Volume) Close() {
v.dataFileAccessLock.Lock()
@@ -100,9 +154,9 @@ func (v *Volume) Close() {
v.nm.Close()
v.nm = nil
}
- if v.dataFile != nil {
- _ = v.dataFile.Close()
- v.dataFile = nil
+ if v.DataBackend != nil {
+ _ = v.DataBackend.Close()
+ v.DataBackend = nil
stats.VolumeServerVolumeCounter.WithLabelValues(v.Collection, "volume").Dec()
}
}
@@ -111,10 +165,6 @@ func (v *Volume) NeedToReplicate() bool {
return v.ReplicaPlacement.GetCopyCount() > 1
}
-func (v *Volume) ContentSize() uint64 {
- return v.nm.ContentSize()
-}
-
// volume is expired if modified time + volume ttl < now
// except when volume is empty
// or when the volume does not have a ttl
@@ -157,18 +207,33 @@ func (v *Volume) expiredLongEnough(maxDelayMinutes uint32) bool {
func (v *Volume) ToVolumeInformationMessage() *master_pb.VolumeInformationMessage {
size, _, modTime := v.FileStat()
- return &master_pb.VolumeInformationMessage{
+
+ volumInfo := &master_pb.VolumeInformationMessage{
Id: uint32(v.Id),
Size: size,
Collection: v.Collection,
- FileCount: uint64(v.nm.FileCount()),
- DeleteCount: uint64(v.nm.DeletedCount()),
- DeletedByteCount: v.nm.DeletedSize(),
- ReadOnly: v.readOnly,
+ FileCount: v.FileCount(),
+ DeleteCount: v.DeletedCount(),
+ DeletedByteCount: v.DeletedSize(),
+ ReadOnly: v.noWriteOrDelete,
ReplicaPlacement: uint32(v.ReplicaPlacement.Byte()),
Version: uint32(v.Version()),
Ttl: v.Ttl.ToUint32(),
CompactRevision: uint32(v.SuperBlock.CompactionRevision),
ModifiedAtSecond: modTime.Unix(),
}
+
+ volumInfo.RemoteStorageName, volumInfo.RemoteStorageKey = v.RemoteStorageNameKey()
+
+ return volumInfo
+}
+
+func (v *Volume) RemoteStorageNameKey() (storageName, storageKey string) {
+ if v.volumeInfo == nil {
+ return
+ }
+ if len(v.volumeInfo.GetFiles()) == 0 {
+ return
+ }
+ return v.volumeInfo.GetFiles()[0].BackendName(), v.volumeInfo.GetFiles()[0].GetKey()
}
diff --git a/weed/storage/volume_backup.go b/weed/storage/volume_backup.go
index f56c40019..ec29c895e 100644
--- a/weed/storage/volume_backup.go
+++ b/weed/storage/volume_backup.go
@@ -6,18 +6,23 @@ import (
"io"
"os"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
- "google.golang.org/grpc"
)
func (v *Volume) GetVolumeSyncStatus() *volume_server_pb.VolumeSyncStatusResponse {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+
var syncStatus = &volume_server_pb.VolumeSyncStatusResponse{}
- if stat, err := v.dataFile.Stat(); err == nil {
- syncStatus.TailOffset = uint64(stat.Size())
+ if datSize, _, err := v.DataBackend.GetStat(); err == nil {
+ syncStatus.TailOffset = uint64(datSize)
}
syncStatus.Collection = v.Collection
syncStatus.IdxFileSize = v.nm.IndexFileSize()
@@ -67,6 +72,8 @@ func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.Dial
return err
}
+ writeOffset := int64(startFromOffset)
+
err = operation.WithVolumeServerClient(volumeServer, grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
stream, err := client.VolumeIncrementalCopy(ctx, &volume_server_pb.VolumeIncrementalCopyRequest{
@@ -77,8 +84,6 @@ func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.Dial
return err
}
- v.dataFile.Seek(int64(startFromOffset), io.SeekStart)
-
for {
resp, recvErr := stream.Recv()
if recvErr != nil {
@@ -89,10 +94,11 @@ func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.Dial
}
}
- _, writeErr := v.dataFile.Write(resp.FileContent)
+ n, writeErr := v.DataBackend.WriteAt(resp.FileContent, writeOffset)
if writeErr != nil {
return writeErr
}
+ writeOffset += int64(n)
}
return nil
@@ -104,7 +110,7 @@ func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.Dial
}
// add to needle map
- return ScanVolumeFileFrom(v.version, v.dataFile, int64(startFromOffset), &VolumeFileScanner4GenIdx{v: v})
+ return ScanVolumeFileFrom(v.Version(), v.DataBackend, int64(startFromOffset), &VolumeFileScanner4GenIdx{v: v})
}
@@ -150,11 +156,11 @@ func (v *Volume) locateLastAppendEntry() (Offset, error) {
func (v *Volume) readAppendAtNs(offset Offset) (uint64, error) {
- n, _, bodyLength, err := needle.ReadNeedleHeader(v.dataFile, v.SuperBlock.version, offset.ToAcutalOffset())
+ n, _, bodyLength, err := needle.ReadNeedleHeader(v.DataBackend, v.SuperBlock.Version, offset.ToAcutalOffset())
if err != nil {
return 0, fmt.Errorf("ReadNeedleHeader: %v", err)
}
- _, err = n.ReadNeedleBody(v.dataFile, v.SuperBlock.version, offset.ToAcutalOffset()+int64(NeedleHeaderSize), bodyLength)
+ _, err = n.ReadNeedleBody(v.DataBackend, v.SuperBlock.Version, offset.ToAcutalOffset()+int64(NeedleHeaderSize), bodyLength)
if err != nil {
return 0, fmt.Errorf("ReadNeedleBody offset %d, bodyLength %d: %v", offset.ToAcutalOffset(), bodyLength, err)
}
@@ -240,7 +246,7 @@ type VolumeFileScanner4GenIdx struct {
v *Volume
}
-func (scanner *VolumeFileScanner4GenIdx) VisitSuperBlock(superBlock SuperBlock) error {
+func (scanner *VolumeFileScanner4GenIdx) VisitSuperBlock(superBlock super_block.SuperBlock) error {
return nil
}
@@ -248,7 +254,7 @@ func (scanner *VolumeFileScanner4GenIdx) ReadNeedleBody() bool {
return false
}
-func (scanner *VolumeFileScanner4GenIdx) VisitNeedle(n *needle.Needle, offset int64) error {
+func (scanner *VolumeFileScanner4GenIdx) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
if n.Size > 0 && n.Size != TombstoneFileSize {
return scanner.v.nm.Put(n.Id, ToOffset(offset), n.Size)
}
diff --git a/weed/storage/volume_checking.go b/weed/storage/volume_checking.go
index 8f930546f..a65c2a3ff 100644
--- a/weed/storage/volume_checking.go
+++ b/weed/storage/volume_checking.go
@@ -4,6 +4,7 @@ import (
"fmt"
"os"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
@@ -29,7 +30,7 @@ func CheckVolumeDataIntegrity(v *Volume, indexFile *os.File) (lastAppendAtNs uin
if size == TombstoneFileSize {
size = 0
}
- if lastAppendAtNs, e = verifyNeedleIntegrity(v.dataFile, v.Version(), offset.ToAcutalOffset(), key, size); e != nil {
+ if lastAppendAtNs, e = verifyNeedleIntegrity(v.DataBackend, v.Version(), offset.ToAcutalOffset(), key, size); e != nil {
return lastAppendAtNs, fmt.Errorf("verifyNeedleIntegrity %s failed: %v", indexFile.Name(), e)
}
return
@@ -54,7 +55,7 @@ func readIndexEntryAtOffset(indexFile *os.File, offset int64) (bytes []byte, err
return
}
-func verifyNeedleIntegrity(datFile *os.File, v needle.Version, offset int64, key NeedleId, size uint32) (lastAppendAtNs uint64, err error) {
+func verifyNeedleIntegrity(datFile backend.BackendStorageFile, v needle.Version, offset int64, key NeedleId, size uint32) (lastAppendAtNs uint64, err error) {
n := new(needle.Needle)
if err = n.ReadData(datFile, offset, size, v); err != nil {
return n.AppendAtNs, err
diff --git a/weed/storage/volume_create.go b/weed/storage/volume_create.go
index 6b3a17439..ffcb246a4 100644
--- a/weed/storage/volume_create.go
+++ b/weed/storage/volume_create.go
@@ -1,4 +1,4 @@
-// +build !linux
+// +build !linux,!windows
package storage
@@ -6,12 +6,16 @@ import (
"os"
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
)
-func createVolumeFile(fileName string, preallocate int64) (file *os.File, e error) {
- file, e = os.OpenFile(fileName, os.O_RDWR|os.O_CREATE, 0644)
+func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.BackendStorageFile, error) {
+ file, e := os.OpenFile(fileName, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0644)
+ if e != nil {
+ return nil, e
+ }
if preallocate > 0 {
glog.V(0).Infof("Preallocated disk space for %s is not supported", fileName)
}
- return file, e
+ return backend.NewDiskFile(file), nil
}
diff --git a/weed/storage/volume_create_linux.go b/weed/storage/volume_create_linux.go
index 8f6bab2fe..ee599ac32 100644
--- a/weed/storage/volume_create_linux.go
+++ b/weed/storage/volume_create_linux.go
@@ -7,13 +7,17 @@ import (
"syscall"
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
)
-func createVolumeFile(fileName string, preallocate int64) (file *os.File, e error) {
- file, e = os.OpenFile(fileName, os.O_RDWR|os.O_CREATE, 0644)
+func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.BackendStorageFile, error) {
+ file, e := os.OpenFile(fileName, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0644)
+ if e != nil {
+ return nil, e
+ }
if preallocate != 0 {
syscall.Fallocate(int(file.Fd()), 1, 0, preallocate)
glog.V(0).Infof("Preallocated %d bytes disk space for %s", preallocate, fileName)
}
- return file, e
+ return backend.NewDiskFile(file), nil
}
diff --git a/weed/storage/volume_create_windows.go b/weed/storage/volume_create_windows.go
new file mode 100644
index 000000000..e1c0b961f
--- /dev/null
+++ b/weed/storage/volume_create_windows.go
@@ -0,0 +1,33 @@
+// +build windows
+
+package storage
+
+import (
+ "github.com/chrislusf/seaweedfs/weed/storage/backend/memory_map"
+ "golang.org/x/sys/windows"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend/memory_map/os_overloads"
+)
+
+func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.BackendStorageFile, error) {
+ if preallocate > 0 {
+ glog.V(0).Infof("Preallocated disk space for %s is not supported", fileName)
+ }
+
+ if memoryMapSizeMB > 0 {
+ file, e := os_overloads.OpenFile(fileName, windows.O_RDWR|windows.O_CREAT, 0644, true)
+ if e != nil {
+ return nil, e
+ }
+ return memory_map.NewMemoryMappedFile(file, memoryMapSizeMB), nil
+ } else {
+ file, e := os_overloads.OpenFile(fileName, windows.O_RDWR|windows.O_CREAT|windows.O_TRUNC, 0644, false)
+ if e != nil {
+ return nil, e
+ }
+ return backend.NewDiskFile(file), nil
+ }
+
+}
diff --git a/weed/storage/volume_info.go b/weed/storage/volume_info.go
index 111058b6e..313818cde 100644
--- a/weed/storage/volume_info.go
+++ b/weed/storage/volume_info.go
@@ -6,37 +6,42 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
type VolumeInfo struct {
- Id needle.VolumeId
- Size uint64
- ReplicaPlacement *ReplicaPlacement
- Ttl *needle.TTL
- Collection string
- Version needle.Version
- FileCount int
- DeleteCount int
- DeletedByteCount uint64
- ReadOnly bool
- CompactRevision uint32
- ModifiedAtSecond int64
+ Id needle.VolumeId
+ Size uint64
+ ReplicaPlacement *super_block.ReplicaPlacement
+ Ttl *needle.TTL
+ Collection string
+ Version needle.Version
+ FileCount int
+ DeleteCount int
+ DeletedByteCount uint64
+ ReadOnly bool
+ CompactRevision uint32
+ ModifiedAtSecond int64
+ RemoteStorageName string
+ RemoteStorageKey string
}
func NewVolumeInfo(m *master_pb.VolumeInformationMessage) (vi VolumeInfo, err error) {
vi = VolumeInfo{
- Id: needle.VolumeId(m.Id),
- Size: m.Size,
- Collection: m.Collection,
- FileCount: int(m.FileCount),
- DeleteCount: int(m.DeleteCount),
- DeletedByteCount: m.DeletedByteCount,
- ReadOnly: m.ReadOnly,
- Version: needle.Version(m.Version),
- CompactRevision: m.CompactRevision,
- ModifiedAtSecond: m.ModifiedAtSecond,
+ Id: needle.VolumeId(m.Id),
+ Size: m.Size,
+ Collection: m.Collection,
+ FileCount: int(m.FileCount),
+ DeleteCount: int(m.DeleteCount),
+ DeletedByteCount: m.DeletedByteCount,
+ ReadOnly: m.ReadOnly,
+ Version: needle.Version(m.Version),
+ CompactRevision: m.CompactRevision,
+ ModifiedAtSecond: m.ModifiedAtSecond,
+ RemoteStorageName: m.RemoteStorageName,
+ RemoteStorageKey: m.RemoteStorageKey,
}
- rp, e := NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
+ rp, e := super_block.NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
if e != nil {
return vi, e
}
@@ -51,7 +56,7 @@ func NewVolumeInfoFromShort(m *master_pb.VolumeShortInformationMessage) (vi Volu
Collection: m.Collection,
Version: needle.Version(m.Version),
}
- rp, e := NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
+ rp, e := super_block.NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
if e != nil {
return vi, e
}
@@ -60,6 +65,10 @@ func NewVolumeInfoFromShort(m *master_pb.VolumeShortInformationMessage) (vi Volu
return vi, nil
}
+func (vi VolumeInfo) IsRemote() bool {
+ return vi.RemoteStorageName != ""
+}
+
func (vi VolumeInfo) String() string {
return fmt.Sprintf("Id:%d, Size:%d, ReplicaPlacement:%s, Collection:%s, Version:%v, FileCount:%d, DeleteCount:%d, DeletedByteCount:%d, ReadOnly:%v",
vi.Id, vi.Size, vi.ReplicaPlacement, vi.Collection, vi.Version, vi.FileCount, vi.DeleteCount, vi.DeletedByteCount, vi.ReadOnly)
@@ -67,18 +76,20 @@ func (vi VolumeInfo) String() string {
func (vi VolumeInfo) ToVolumeInformationMessage() *master_pb.VolumeInformationMessage {
return &master_pb.VolumeInformationMessage{
- Id: uint32(vi.Id),
- Size: uint64(vi.Size),
- Collection: vi.Collection,
- FileCount: uint64(vi.FileCount),
- DeleteCount: uint64(vi.DeleteCount),
- DeletedByteCount: vi.DeletedByteCount,
- ReadOnly: vi.ReadOnly,
- ReplicaPlacement: uint32(vi.ReplicaPlacement.Byte()),
- Version: uint32(vi.Version),
- Ttl: vi.Ttl.ToUint32(),
- CompactRevision: vi.CompactRevision,
- ModifiedAtSecond: vi.ModifiedAtSecond,
+ Id: uint32(vi.Id),
+ Size: uint64(vi.Size),
+ Collection: vi.Collection,
+ FileCount: uint64(vi.FileCount),
+ DeleteCount: uint64(vi.DeleteCount),
+ DeletedByteCount: vi.DeletedByteCount,
+ ReadOnly: vi.ReadOnly,
+ ReplicaPlacement: uint32(vi.ReplicaPlacement.Byte()),
+ Version: uint32(vi.Version),
+ Ttl: vi.Ttl.ToUint32(),
+ CompactRevision: vi.CompactRevision,
+ ModifiedAtSecond: vi.ModifiedAtSecond,
+ RemoteStorageName: vi.RemoteStorageName,
+ RemoteStorageKey: vi.RemoteStorageKey,
}
}
diff --git a/weed/storage/volume_loading.go b/weed/storage/volume_loading.go
index ed1cfe124..fa1f7d617 100644
--- a/weed/storage/volume_loading.go
+++ b/weed/storage/volume_loading.go
@@ -3,139 +3,145 @@ package storage
import (
"fmt"
"os"
- "time"
- "github.com/chrislusf/seaweedfs/weed/stats"
- "github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/syndtr/goleveldb/leveldb/opt"
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+ "github.com/chrislusf/seaweedfs/weed/util"
)
-func loadVolumeWithoutIndex(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType) (v *Volume, e error) {
+func loadVolumeWithoutIndex(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType) (v *Volume, err error) {
v = &Volume{dir: dirname, Collection: collection, Id: id}
- v.SuperBlock = SuperBlock{}
+ v.SuperBlock = super_block.SuperBlock{}
v.needleMapKind = needleMapKind
- e = v.load(false, false, needleMapKind, 0)
+ err = v.load(false, false, needleMapKind, 0)
return
}
-func (v *Volume) load(alsoLoadIndex bool, createDatIfMissing bool, needleMapKind NeedleMapType, preallocate int64) error {
- var e error
+func (v *Volume) load(alsoLoadIndex bool, createDatIfMissing bool, needleMapKind NeedleMapType, preallocate int64) (err error) {
fileName := v.FileName()
alreadyHasSuperBlock := false
- if exists, canRead, canWrite, modifiedTime, fileSize := checkFile(fileName + ".dat"); exists {
+ if !v.maybeLoadVolumeInfo() {
+ v.SaveVolumeInfo()
+ }
+
+ if v.HasRemoteFile() {
+ v.noWriteCanDelete = true
+ v.noWriteOrDelete = false
+ glog.V(0).Infof("loading volume %d from remote %v", v.Id, v.volumeInfo.Files)
+ v.LoadRemoteFile()
+ alreadyHasSuperBlock = true
+ } else if exists, canRead, canWrite, modifiedTime, fileSize := util.CheckFile(fileName + ".dat"); exists {
+ // open dat file
if !canRead {
return fmt.Errorf("cannot read Volume Data file %s.dat", fileName)
}
+ var dataFile *os.File
if canWrite {
- v.dataFile, e = os.OpenFile(fileName+".dat", os.O_RDWR|os.O_CREATE, 0644)
- v.lastModifiedTsSeconds = uint64(modifiedTime.Unix())
+ dataFile, err = os.OpenFile(fileName+".dat", os.O_RDWR|os.O_CREATE, 0644)
} else {
glog.V(0).Infoln("opening " + fileName + ".dat in READONLY mode")
- v.dataFile, e = os.Open(fileName + ".dat")
- v.readOnly = true
+ dataFile, err = os.Open(fileName + ".dat")
+ v.noWriteOrDelete = true
}
- if fileSize >= _SuperBlockSize {
+ v.lastModifiedTsSeconds = uint64(modifiedTime.Unix())
+ if fileSize >= super_block.SuperBlockSize {
alreadyHasSuperBlock = true
}
+ v.DataBackend = backend.NewDiskFile(dataFile)
} else {
if createDatIfMissing {
- v.dataFile, e = createVolumeFile(fileName+".dat", preallocate)
+ v.DataBackend, err = createVolumeFile(fileName+".dat", preallocate, v.MemoryMapMaxSizeMb)
} else {
return fmt.Errorf("Volume Data file %s.dat does not exist.", fileName)
}
}
- if e != nil {
- if !os.IsPermission(e) {
- return fmt.Errorf("cannot load Volume Data %s.dat: %v", fileName, e)
+ if err != nil {
+ if !os.IsPermission(err) {
+ return fmt.Errorf("cannot load Volume Data %s.dat: %v", fileName, err)
} else {
- return fmt.Errorf("load data file %s.dat: %v", fileName, e)
+ return fmt.Errorf("load data file %s.dat: %v", fileName, err)
}
}
if alreadyHasSuperBlock {
- e = v.readSuperBlock()
+ err = v.readSuperBlock()
} else {
- e = v.maybeWriteSuperBlock()
+ if !v.SuperBlock.Initialized() {
+ return fmt.Errorf("volume %s.dat not initialized", fileName)
+ }
+ err = v.maybeWriteSuperBlock()
}
- if e == nil && alsoLoadIndex {
+ if err == nil && alsoLoadIndex {
var indexFile *os.File
- if v.readOnly {
- glog.V(1).Infoln("open to read file", fileName+".idx")
- if indexFile, e = os.OpenFile(fileName+".idx", os.O_RDONLY, 0644); e != nil {
- return fmt.Errorf("cannot read Volume Index %s.idx: %v", fileName, e)
+ if v.noWriteOrDelete {
+ glog.V(0).Infoln("open to read file", fileName+".idx")
+ if indexFile, err = os.OpenFile(fileName+".idx", os.O_RDONLY, 0644); err != nil {
+ return fmt.Errorf("cannot read Volume Index %s.idx: %v", fileName, err)
}
} else {
glog.V(1).Infoln("open to write file", fileName+".idx")
- if indexFile, e = os.OpenFile(fileName+".idx", os.O_RDWR|os.O_CREATE, 0644); e != nil {
- return fmt.Errorf("cannot write Volume Index %s.idx: %v", fileName, e)
+ if indexFile, err = os.OpenFile(fileName+".idx", os.O_RDWR|os.O_CREATE, 0644); err != nil {
+ return fmt.Errorf("cannot write Volume Index %s.idx: %v", fileName, err)
}
}
- if v.lastAppendAtNs, e = CheckVolumeDataIntegrity(v, indexFile); e != nil {
- v.readOnly = true
- glog.V(0).Infof("volumeDataIntegrityChecking failed %v", e)
+ if v.lastAppendAtNs, err = CheckVolumeDataIntegrity(v, indexFile); err != nil {
+ v.noWriteOrDelete = true
+ glog.V(0).Infof("volumeDataIntegrityChecking failed %v", err)
}
- switch needleMapKind {
- case NeedleMapInMemory:
- glog.V(0).Infoln("loading index", fileName+".idx", "to memory readonly", v.readOnly)
- if v.nm, e = LoadCompactNeedleMap(indexFile); e != nil {
- glog.V(0).Infof("loading index %s to memory error: %v", fileName+".idx", e)
- }
- case NeedleMapLevelDb:
- glog.V(0).Infoln("loading leveldb", fileName+".ldb")
- opts := &opt.Options{
- BlockCacheCapacity: 2 * 1024 * 1024, // default value is 8MiB
- WriteBuffer: 1 * 1024 * 1024, // default value is 4MiB
- CompactionTableSizeMultiplier: 10, // default value is 1
- }
- if v.nm, e = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); e != nil {
- glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", e)
- }
- case NeedleMapLevelDbMedium:
- glog.V(0).Infoln("loading leveldb medium", fileName+".ldb")
- opts := &opt.Options{
- BlockCacheCapacity: 4 * 1024 * 1024, // default value is 8MiB
- WriteBuffer: 2 * 1024 * 1024, // default value is 4MiB
- CompactionTableSizeMultiplier: 10, // default value is 1
- }
- if v.nm, e = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); e != nil {
- glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", e)
- }
- case NeedleMapLevelDbLarge:
- glog.V(0).Infoln("loading leveldb large", fileName+".ldb")
- opts := &opt.Options{
- BlockCacheCapacity: 8 * 1024 * 1024, // default value is 8MiB
- WriteBuffer: 4 * 1024 * 1024, // default value is 4MiB
- CompactionTableSizeMultiplier: 10, // default value is 1
+
+ if v.noWriteOrDelete || v.noWriteCanDelete {
+ if v.nm, err = NewSortedFileNeedleMap(fileName, indexFile); err != nil {
+ glog.V(0).Infof("loading sorted db %s error: %v", fileName+".sdx", err)
}
- if v.nm, e = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); e != nil {
- glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", e)
+ } else {
+ switch needleMapKind {
+ case NeedleMapInMemory:
+ glog.V(0).Infoln("loading index", fileName+".idx", "to memory")
+ if v.nm, err = LoadCompactNeedleMap(indexFile); err != nil {
+ glog.V(0).Infof("loading index %s to memory error: %v", fileName+".idx", err)
+ }
+ case NeedleMapLevelDb:
+ glog.V(0).Infoln("loading leveldb", fileName+".ldb")
+ opts := &opt.Options{
+ BlockCacheCapacity: 2 * 1024 * 1024, // default value is 8MiB
+ WriteBuffer: 1 * 1024 * 1024, // default value is 4MiB
+ CompactionTableSizeMultiplier: 10, // default value is 1
+ }
+ if v.nm, err = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); err != nil {
+ glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", err)
+ }
+ case NeedleMapLevelDbMedium:
+ glog.V(0).Infoln("loading leveldb medium", fileName+".ldb")
+ opts := &opt.Options{
+ BlockCacheCapacity: 4 * 1024 * 1024, // default value is 8MiB
+ WriteBuffer: 2 * 1024 * 1024, // default value is 4MiB
+ CompactionTableSizeMultiplier: 10, // default value is 1
+ }
+ if v.nm, err = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); err != nil {
+ glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", err)
+ }
+ case NeedleMapLevelDbLarge:
+ glog.V(0).Infoln("loading leveldb large", fileName+".ldb")
+ opts := &opt.Options{
+ BlockCacheCapacity: 8 * 1024 * 1024, // default value is 8MiB
+ WriteBuffer: 4 * 1024 * 1024, // default value is 4MiB
+ CompactionTableSizeMultiplier: 10, // default value is 1
+ }
+ if v.nm, err = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); err != nil {
+ glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", err)
+ }
}
}
}
stats.VolumeServerVolumeCounter.WithLabelValues(v.Collection, "volume").Inc()
- return e
-}
-
-func checkFile(filename string) (exists, canRead, canWrite bool, modTime time.Time, fileSize int64) {
- exists = true
- fi, err := os.Stat(filename)
- if os.IsNotExist(err) {
- exists = false
- return
- }
- if fi.Mode()&0400 != 0 {
- canRead = true
- }
- if fi.Mode()&0200 != 0 {
- canWrite = true
- }
- modTime = fi.ModTime()
- fileSize = fi.Size()
- return
+ return err
}
diff --git a/weed/storage/volume_read_write.go b/weed/storage/volume_read_write.go
index d080e5dba..0aa3f794b 100644
--- a/weed/storage/volume_read_write.go
+++ b/weed/storage/volume_read_write.go
@@ -9,7 +9,9 @@ import (
"time"
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -21,10 +23,11 @@ func (v *Volume) isFileUnchanged(n *needle.Needle) bool {
if v.Ttl.String() != "" {
return false
}
+
nv, ok := v.nm.Get(n.Id)
if ok && !nv.Offset.IsZero() && nv.Size != TombstoneFileSize {
oldNeedle := new(needle.Needle)
- err := oldNeedle.ReadData(v.dataFile, nv.Offset.ToAcutalOffset(), nv.Size, v.Version())
+ err := oldNeedle.ReadData(v.DataBackend, nv.Offset.ToAcutalOffset(), nv.Size, v.Version())
if err != nil {
glog.V(0).Infof("Failed to check updated file at offset %d size %d: %v", nv.Offset.ToAcutalOffset(), nv.Size, err)
return false
@@ -39,26 +42,29 @@ func (v *Volume) isFileUnchanged(n *needle.Needle) bool {
// Destroy removes everything related to this volume
func (v *Volume) Destroy() (err error) {
- if v.readOnly {
- err = fmt.Errorf("%s is read-only", v.dataFile.Name())
+ if v.isCompacting {
+ err = fmt.Errorf("volume %d is compacting", v.Id)
return
}
+ storageName, storageKey := v.RemoteStorageNameKey()
+ if v.HasRemoteFile() && storageName != "" && storageKey != "" {
+ if backendStorage, found := backend.BackendStorages[storageName]; found {
+ backendStorage.DeleteFile(storageKey)
+ }
+ }
v.Close()
os.Remove(v.FileName() + ".dat")
os.Remove(v.FileName() + ".idx")
+ os.Remove(v.FileName() + ".vif")
+ os.Remove(v.FileName() + ".sdx")
os.Remove(v.FileName() + ".cpd")
os.Remove(v.FileName() + ".cpx")
- os.Remove(v.FileName() + ".ldb")
- os.Remove(v.FileName() + ".bdb")
+ os.RemoveAll(v.FileName() + ".ldb")
return
}
func (v *Volume) writeNeedle(n *needle.Needle) (offset uint64, size uint32, isUnchanged bool, err error) {
glog.V(4).Infof("writing needle %s", needle.NewFileIdFromNeedle(v.Id, n).String())
- if v.readOnly {
- err = fmt.Errorf("%s is read-only", v.dataFile.Name())
- return
- }
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
if v.isFileUnchanged(n) {
@@ -75,7 +81,7 @@ func (v *Volume) writeNeedle(n *needle.Needle) (offset uint64, size uint32, isUn
// check whether existing needle cookie matches
nv, ok := v.nm.Get(n.Id)
if ok {
- existingNeedle, _, _, existingNeedleReadErr := needle.ReadNeedleHeader(v.dataFile, v.Version(), nv.Offset.ToAcutalOffset())
+ existingNeedle, _, _, existingNeedleReadErr := needle.ReadNeedleHeader(v.DataBackend, v.Version(), nv.Offset.ToAcutalOffset())
if existingNeedleReadErr != nil {
err = fmt.Errorf("reading existing needle: %v", existingNeedleReadErr)
return
@@ -89,7 +95,7 @@ func (v *Volume) writeNeedle(n *needle.Needle) (offset uint64, size uint32, isUn
// append to dat file
n.AppendAtNs = uint64(time.Now().UnixNano())
- if offset, size, _, err = n.Append(v.dataFile, v.Version()); err != nil {
+ if offset, size, _, err = n.Append(v.DataBackend, v.Version()); err != nil {
return
}
v.lastAppendAtNs = n.AppendAtNs
@@ -108,9 +114,6 @@ func (v *Volume) writeNeedle(n *needle.Needle) (offset uint64, size uint32, isUn
func (v *Volume) deleteNeedle(n *needle.Needle) (uint32, error) {
glog.V(4).Infof("delete needle %s", needle.NewFileIdFromNeedle(v.Id, n).String())
- if v.readOnly {
- return 0, fmt.Errorf("%s is read-only", v.dataFile.Name())
- }
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
nv, ok := v.nm.Get(n.Id)
@@ -119,7 +122,7 @@ func (v *Volume) deleteNeedle(n *needle.Needle) (uint32, error) {
size := nv.Size
n.Data = nil
n.AppendAtNs = uint64(time.Now().UnixNano())
- offset, _, _, err := n.Append(v.dataFile, v.Version())
+ offset, _, _, err := n.Append(v.DataBackend, v.Version())
if err != nil {
return size, err
}
@@ -134,13 +137,12 @@ func (v *Volume) deleteNeedle(n *needle.Needle) (uint32, error) {
// read fills in Needle content by looking up n.Id from NeedleMapper
func (v *Volume) readNeedle(n *needle.Needle) (int, error) {
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
+
nv, ok := v.nm.Get(n.Id)
if !ok || nv.Offset.IsZero() {
- v.compactingWg.Wait()
- nv, ok = v.nm.Get(n.Id)
- if !ok || nv.Offset.IsZero() {
- return -1, ErrorNotFound
- }
+ return -1, ErrorNotFound
}
if nv.Size == TombstoneFileSize {
return -1, errors.New("already deleted")
@@ -148,7 +150,7 @@ func (v *Volume) readNeedle(n *needle.Needle) (int, error) {
if nv.Size == 0 {
return 0, nil
}
- err := n.ReadData(v.dataFile, nv.Offset.ToAcutalOffset(), nv.Size, v.Version())
+ err := n.ReadData(v.DataBackend, nv.Offset.ToAcutalOffset(), nv.Size, v.Version())
if err != nil {
return 0, err
}
@@ -170,9 +172,9 @@ func (v *Volume) readNeedle(n *needle.Needle) (int, error) {
}
type VolumeFileScanner interface {
- VisitSuperBlock(SuperBlock) error
+ VisitSuperBlock(super_block.SuperBlock) error
ReadNeedleBody() bool
- VisitNeedle(n *needle.Needle, offset int64) error
+ VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error
}
func ScanVolumeFile(dirname string, collection string, id needle.VolumeId,
@@ -182,8 +184,10 @@ func ScanVolumeFile(dirname string, collection string, id needle.VolumeId,
if v, err = loadVolumeWithoutIndex(dirname, collection, id, needleMapKind); err != nil {
return fmt.Errorf("failed to load volume %d: %v", id, err)
}
- if err = volumeFileScanner.VisitSuperBlock(v.SuperBlock); err != nil {
- return fmt.Errorf("failed to process volume %d super block: %v", id, err)
+ if v.volumeInfo.Version == 0 {
+ if err = volumeFileScanner.VisitSuperBlock(v.SuperBlock); err != nil {
+ return fmt.Errorf("failed to process volume %d super block: %v", id, err)
+ }
}
defer v.Close()
@@ -191,26 +195,27 @@ func ScanVolumeFile(dirname string, collection string, id needle.VolumeId,
offset := int64(v.SuperBlock.BlockSize())
- return ScanVolumeFileFrom(version, v.dataFile, offset, volumeFileScanner)
+ return ScanVolumeFileFrom(version, v.DataBackend, offset, volumeFileScanner)
}
-func ScanVolumeFileFrom(version needle.Version, dataFile *os.File, offset int64, volumeFileScanner VolumeFileScanner) (err error) {
- n, _, rest, e := needle.ReadNeedleHeader(dataFile, version, offset)
+func ScanVolumeFileFrom(version needle.Version, datBackend backend.BackendStorageFile, offset int64, volumeFileScanner VolumeFileScanner) (err error) {
+ n, nh, rest, e := needle.ReadNeedleHeader(datBackend, version, offset)
if e != nil {
if e == io.EOF {
return nil
}
- return fmt.Errorf("cannot read %s at offset %d: %v", dataFile.Name(), offset, e)
+ return fmt.Errorf("cannot read %s at offset %d: %v", datBackend.Name(), offset, e)
}
for n != nil {
+ var needleBody []byte
if volumeFileScanner.ReadNeedleBody() {
- if _, err = n.ReadNeedleBody(dataFile, version, offset+NeedleHeaderSize, rest); err != nil {
+ if needleBody, err = n.ReadNeedleBody(datBackend, version, offset+NeedleHeaderSize, rest); err != nil {
glog.V(0).Infof("cannot read needle body: %v", err)
//err = fmt.Errorf("cannot read needle body: %v", err)
//return
}
}
- err := volumeFileScanner.VisitNeedle(n, offset)
+ err := volumeFileScanner.VisitNeedle(n, offset, nh, needleBody)
if err == io.EOF {
return nil
}
@@ -220,40 +225,7 @@ func ScanVolumeFileFrom(version needle.Version, dataFile *os.File, offset int64,
}
offset += NeedleHeaderSize + rest
glog.V(4).Infof("==> new entry offset %d", offset)
- if n, _, rest, err = needle.ReadNeedleHeader(dataFile, version, offset); err != nil {
- if err == io.EOF {
- return nil
- }
- return fmt.Errorf("cannot read needle header at offset %d: %v", offset, err)
- }
- glog.V(4).Infof("new entry needle size:%d rest:%d", n.Size, rest)
- }
- return nil
-}
-
-func ScanVolumeFileNeedleFrom(version needle.Version, dataFile *os.File, offset int64, fn func(needleHeader, needleBody []byte, needleAppendAtNs uint64) error) (err error) {
- n, nh, rest, e := needle.ReadNeedleHeader(dataFile, version, offset)
- if e != nil {
- if e == io.EOF {
- return nil
- }
- return fmt.Errorf("cannot read %s at offset %d: %v", dataFile.Name(), offset, e)
- }
- for n != nil {
- var needleBody []byte
- if needleBody, err = n.ReadNeedleBody(dataFile, version, offset+NeedleHeaderSize, rest); err != nil {
- glog.V(0).Infof("cannot read needle body: %v", err)
- //err = fmt.Errorf("cannot read needle body: %v", err)
- //return
- }
- err = fn(nh, needleBody, n.AppendAtNs)
- if err != nil {
- glog.V(0).Infof("visit needle error: %v", err)
- return
- }
- offset += NeedleHeaderSize + rest
- glog.V(4).Infof("==> new entry offset %d", offset)
- if n, nh, rest, err = needle.ReadNeedleHeader(dataFile, version, offset); err != nil {
+ if n, nh, rest, err = needle.ReadNeedleHeader(datBackend, version, offset); err != nil {
if err == io.EOF {
return nil
}
diff --git a/weed/storage/volume_super_block.go b/weed/storage/volume_super_block.go
index 164c887e1..61c09d85a 100644
--- a/weed/storage/volume_super_block.go
+++ b/weed/storage/volume_super_block.go
@@ -1,88 +1,32 @@
package storage
import (
- "fmt"
"os"
"github.com/chrislusf/seaweedfs/weed/glog"
- "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
- "github.com/chrislusf/seaweedfs/weed/util"
- "github.com/golang/protobuf/proto"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
-const (
- _SuperBlockSize = 8
-)
-
-/*
-* Super block currently has 8 bytes allocated for each volume.
-* Byte 0: version, 1 or 2
-* Byte 1: Replica Placement strategy, 000, 001, 002, 010, etc
-* Byte 2 and byte 3: Time to live. See TTL for definition
-* Byte 4 and byte 5: The number of times the volume has been compacted.
-* Rest bytes: Reserved
- */
-type SuperBlock struct {
- version needle.Version
- ReplicaPlacement *ReplicaPlacement
- Ttl *needle.TTL
- CompactionRevision uint16
- Extra *master_pb.SuperBlockExtra
- extraSize uint16
-}
-
-func (s *SuperBlock) BlockSize() int {
- switch s.version {
- case needle.Version2, needle.Version3:
- return _SuperBlockSize + int(s.extraSize)
- }
- return _SuperBlockSize
-}
-
-func (s *SuperBlock) Version() needle.Version {
- return s.version
-}
-func (s *SuperBlock) Bytes() []byte {
- header := make([]byte, _SuperBlockSize)
- header[0] = byte(s.version)
- header[1] = s.ReplicaPlacement.Byte()
- s.Ttl.ToBytes(header[2:4])
- util.Uint16toBytes(header[4:6], s.CompactionRevision)
-
- if s.Extra != nil {
- extraData, err := proto.Marshal(s.Extra)
- if err != nil {
- glog.Fatalf("cannot marshal super block extra %+v: %v", s.Extra, err)
- }
- extraSize := len(extraData)
- if extraSize > 256*256-2 {
- // reserve a couple of bits for future extension
- glog.Fatalf("super block extra size is %d bigger than %d", extraSize, 256*256-2)
- }
- s.extraSize = uint16(extraSize)
- util.Uint16toBytes(header[6:8], s.extraSize)
-
- header = append(header, extraData...)
- }
-
- return header
-}
-
func (v *Volume) maybeWriteSuperBlock() error {
- stat, e := v.dataFile.Stat()
+
+ datSize, _, e := v.DataBackend.GetStat()
if e != nil {
- glog.V(0).Infof("failed to stat datafile %s: %v", v.dataFile.Name(), e)
+ glog.V(0).Infof("failed to stat datafile %s: %v", v.DataBackend.Name(), e)
return e
}
- if stat.Size() == 0 {
- v.SuperBlock.version = needle.CurrentVersion
- _, e = v.dataFile.Write(v.SuperBlock.Bytes())
+ if datSize == 0 {
+ v.SuperBlock.Version = needle.CurrentVersion
+ _, e = v.DataBackend.WriteAt(v.SuperBlock.Bytes(), 0)
if e != nil && os.IsPermission(e) {
//read-only, but zero length - recreate it!
- if v.dataFile, e = os.Create(v.dataFile.Name()); e == nil {
- if _, e = v.dataFile.Write(v.SuperBlock.Bytes()); e == nil {
- v.readOnly = false
+ var dataFile *os.File
+ if dataFile, e = os.Create(v.DataBackend.Name()); e == nil {
+ v.DataBackend = backend.NewDiskFile(dataFile)
+ if _, e = v.DataBackend.WriteAt(v.SuperBlock.Bytes(), 0); e == nil {
+ v.noWriteOrDelete = false
+ v.noWriteCanDelete = false
}
}
}
@@ -91,40 +35,6 @@ func (v *Volume) maybeWriteSuperBlock() error {
}
func (v *Volume) readSuperBlock() (err error) {
- v.SuperBlock, err = ReadSuperBlock(v.dataFile)
+ v.SuperBlock, err = super_block.ReadSuperBlock(v.DataBackend)
return err
}
-
-// ReadSuperBlock reads from data file and load it into volume's super block
-func ReadSuperBlock(dataFile *os.File) (superBlock SuperBlock, err error) {
- if _, err = dataFile.Seek(0, 0); err != nil {
- err = fmt.Errorf("cannot seek to the beginning of %s: %v", dataFile.Name(), err)
- return
- }
- header := make([]byte, _SuperBlockSize)
- if _, e := dataFile.Read(header); e != nil {
- err = fmt.Errorf("cannot read volume %s super block: %v", dataFile.Name(), e)
- return
- }
- superBlock.version = needle.Version(header[0])
- if superBlock.ReplicaPlacement, err = NewReplicaPlacementFromByte(header[1]); err != nil {
- err = fmt.Errorf("cannot read replica type: %s", err.Error())
- return
- }
- superBlock.Ttl = needle.LoadTTLFromBytes(header[2:4])
- superBlock.CompactionRevision = util.BytesToUint16(header[4:6])
- superBlock.extraSize = util.BytesToUint16(header[6:8])
-
- if superBlock.extraSize > 0 {
- // read more
- extraData := make([]byte, int(superBlock.extraSize))
- superBlock.Extra = &master_pb.SuperBlockExtra{}
- err = proto.Unmarshal(extraData, superBlock.Extra)
- if err != nil {
- err = fmt.Errorf("cannot read volume %s super block extra: %v", dataFile.Name(), err)
- return
- }
- }
-
- return
-}
diff --git a/weed/storage/volume_tier.go b/weed/storage/volume_tier.go
new file mode 100644
index 000000000..99071285f
--- /dev/null
+++ b/weed/storage/volume_tier.go
@@ -0,0 +1,51 @@
+package storage
+
+import (
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ _ "github.com/chrislusf/seaweedfs/weed/storage/backend/s3_backend"
+)
+
+func (v *Volume) GetVolumeInfo() *volume_server_pb.VolumeInfo {
+ return v.volumeInfo
+}
+
+func (v *Volume) maybeLoadVolumeInfo() (found bool) {
+
+ v.volumeInfo, found = pb.MaybeLoadVolumeInfo(v.FileName() + ".vif")
+
+ if found {
+ glog.V(0).Infof("volume %d is tiered to %s as %s and read only", v.Id,
+ v.volumeInfo.Files[0].BackendName(), v.volumeInfo.Files[0].Key)
+ v.hasRemoteFile = true
+ }
+
+ return
+
+}
+
+func (v *Volume) HasRemoteFile() bool {
+ return v.hasRemoteFile
+}
+
+func (v *Volume) LoadRemoteFile() error {
+ tierFile := v.volumeInfo.GetFiles()[0]
+ backendStorage := backend.BackendStorages[tierFile.BackendName()]
+
+ if v.DataBackend != nil {
+ v.DataBackend.Close()
+ }
+
+ v.DataBackend = backendStorage.NewStorageFile(tierFile.Key, v.volumeInfo)
+ return nil
+}
+
+func (v *Volume) SaveVolumeInfo() error {
+
+ tierFileName := v.FileName() + ".vif"
+
+ return pb.SaveVolumeInfo(tierFileName, v.volumeInfo)
+
+}
diff --git a/weed/storage/volume_vacuum.go b/weed/storage/volume_vacuum.go
index 3bb306649..434b5989d 100644
--- a/weed/storage/volume_vacuum.go
+++ b/weed/storage/volume_vacuum.go
@@ -7,8 +7,11 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
idx2 "github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -17,42 +20,81 @@ func (v *Volume) garbageLevel() float64 {
if v.ContentSize() == 0 {
return 0
}
- return float64(v.nm.DeletedSize()) / float64(v.ContentSize())
+ deletedSize := v.DeletedSize()
+ fileSize := v.ContentSize()
+ if v.DeletedCount() > 0 && v.DeletedSize() == 0 {
+ // this happens for .sdx converted back to normal .idx
+ // where deleted entry size is missing
+ datFileSize, _, _ := v.FileStat()
+ deletedSize = datFileSize - fileSize - super_block.SuperBlockSize
+ fileSize = datFileSize
+ }
+ return float64(deletedSize) / float64(fileSize)
}
+// compact a volume based on deletions in .dat files
func (v *Volume) Compact(preallocate int64, compactionBytePerSecond int64) error {
+
+ if v.MemoryMapMaxSizeMb != 0 { //it makes no sense to compact in memory
+ return nil
+ }
glog.V(3).Infof("Compacting volume %d ...", v.Id)
//no need to lock for copy on write
//v.accessLock.Lock()
//defer v.accessLock.Unlock()
//glog.V(3).Infof("Got Compaction lock...")
+ v.isCompacting = true
+ defer func() {
+ v.isCompacting = false
+ }()
filePath := v.FileName()
- v.lastCompactIndexOffset = v.nm.IndexFileSize()
+ v.lastCompactIndexOffset = v.IndexFileSize()
v.lastCompactRevision = v.SuperBlock.CompactionRevision
glog.V(3).Infof("creating copies for volume %d ,last offset %d...", v.Id, v.lastCompactIndexOffset)
return v.copyDataAndGenerateIndexFile(filePath+".cpd", filePath+".cpx", preallocate, compactionBytePerSecond)
}
-func (v *Volume) Compact2() error {
+// compact a volume based on deletions in .idx files
+func (v *Volume) Compact2(preallocate int64) error {
+
+ if v.MemoryMapMaxSizeMb != 0 { //it makes no sense to compact in memory
+ return nil
+ }
glog.V(3).Infof("Compact2 volume %d ...", v.Id)
+
+ v.isCompacting = true
+ defer func() {
+ v.isCompacting = false
+ }()
+
filePath := v.FileName()
+ v.lastCompactIndexOffset = v.IndexFileSize()
+ v.lastCompactRevision = v.SuperBlock.CompactionRevision
glog.V(3).Infof("creating copies for volume %d ...", v.Id)
- return v.copyDataBasedOnIndexFile(filePath+".cpd", filePath+".cpx")
+ return v.copyDataBasedOnIndexFile(filePath+".cpd", filePath+".cpx", preallocate)
}
func (v *Volume) CommitCompact() error {
+ if v.MemoryMapMaxSizeMb != 0 { //it makes no sense to compact in memory
+ return nil
+ }
glog.V(0).Infof("Committing volume %d vacuuming...", v.Id)
+
+ v.isCompacting = true
+ defer func() {
+ v.isCompacting = false
+ }()
+
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
+
glog.V(3).Infof("Got volume %d committing lock...", v.Id)
- v.compactingWg.Add(1)
- defer v.compactingWg.Done()
v.nm.Close()
- if err := v.dataFile.Close(); err != nil {
+ if err := v.DataBackend.Close(); err != nil {
glog.V(0).Infof("fail to close volume %d", v.Id)
}
- v.dataFile = nil
+ v.DataBackend = nil
stats.VolumeServerVolumeCounter.WithLabelValues(v.Collection, "volume").Dec()
var e error
@@ -103,14 +145,15 @@ func (v *Volume) cleanupCompact() error {
return nil
}
-func fetchCompactRevisionFromDatFile(file *os.File) (compactRevision uint16, err error) {
- superBlock, err := ReadSuperBlock(file)
+func fetchCompactRevisionFromDatFile(datBackend backend.BackendStorageFile) (compactRevision uint16, err error) {
+ superBlock, err := super_block.ReadSuperBlock(datBackend)
if err != nil {
return 0, err
}
return superBlock.CompactionRevision, nil
}
+// if old .dat and .idx files are updated, this func tries to apply the same changes to new files accordingly
func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldIdxFileName string) (err error) {
var indexSize int64
@@ -118,8 +161,10 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
defer oldIdxFile.Close()
oldDatFile, err := os.Open(oldDatFileName)
- defer oldDatFile.Close()
+ oldDatBackend := backend.NewDiskFile(oldDatFile)
+ defer oldDatBackend.Close()
+ // skip if the old .idx file has not changed
if indexSize, err = verifyIndexFileIntegrity(oldIdxFile); err != nil {
return fmt.Errorf("verifyIndexFileIntegrity %s failed: %v", oldIdxFileName, err)
}
@@ -127,7 +172,8 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
return nil
}
- oldDatCompactRevision, err := fetchCompactRevisionFromDatFile(oldDatFile)
+ // fail if the old .dat file has changed to a new revision
+ oldDatCompactRevision, err := fetchCompactRevisionFromDatFile(oldDatBackend)
if err != nil {
return fmt.Errorf("fetchCompactRevisionFromDatFile src %s failed: %v", oldDatFile.Name(), err)
}
@@ -168,7 +214,8 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
if dst, err = os.OpenFile(newDatFileName, os.O_RDWR, 0644); err != nil {
return fmt.Errorf("open dat file %s failed: %v", newDatFileName, err)
}
- defer dst.Close()
+ dstDatBackend := backend.NewDiskFile(dst)
+ defer dstDatBackend.Close()
if idx, err = os.OpenFile(newIdxFileName, os.O_RDWR, 0644); err != nil {
return fmt.Errorf("open idx file %s failed: %v", newIdxFileName, err)
@@ -176,7 +223,7 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
defer idx.Close()
var newDatCompactRevision uint16
- newDatCompactRevision, err = fetchCompactRevisionFromDatFile(dst)
+ newDatCompactRevision, err = fetchCompactRevisionFromDatFile(dstDatBackend)
if err != nil {
return fmt.Errorf("fetchCompactRevisionFromDatFile dst %s failed: %v", dst.Name(), err)
}
@@ -184,11 +231,9 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
return fmt.Errorf("oldDatFile %s 's compact revision is %d while newDatFile %s 's compact revision is %d", oldDatFileName, oldDatCompactRevision, newDatFileName, newDatCompactRevision)
}
- idxEntryBytes := make([]byte, NeedleIdSize+OffsetSize+SizeSize)
for key, increIdxEntry := range incrementedHasUpdatedIndexEntry {
- NeedleIdToBytes(idxEntryBytes[0:NeedleIdSize], key)
- OffsetToBytes(idxEntryBytes[NeedleIdSize:NeedleIdSize+OffsetSize], increIdxEntry.offset)
- util.Uint32toBytes(idxEntryBytes[NeedleIdSize+OffsetSize:NeedleIdSize+OffsetSize+SizeSize], increIdxEntry.size)
+
+ idxEntryBytes := needle_map.ToBytes(key, increIdxEntry.offset, increIdxEntry.size)
var offset int64
if offset, err = dst.Seek(0, 2); err != nil {
@@ -198,8 +243,8 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
//ensure file writing starting from aligned positions
if offset%NeedlePaddingSize != 0 {
offset = offset + (NeedlePaddingSize - offset%NeedlePaddingSize)
- if offset, err = v.dataFile.Seek(offset, 0); err != nil {
- glog.V(0).Infof("failed to align in datafile %s: %v", v.dataFile.Name(), err)
+ if offset, err = dst.Seek(offset, 0); err != nil {
+ glog.V(0).Infof("failed to align in datafile %s: %v", dst.Name(), err)
return
}
}
@@ -209,7 +254,7 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
//even the needle cache in memory is hit, the need_bytes is correct
glog.V(4).Infof("file %d offset %d size %d", key, increIdxEntry.offset.ToAcutalOffset(), increIdxEntry.size)
var needleBytes []byte
- needleBytes, err = needle.ReadNeedleBlob(oldDatFile, increIdxEntry.offset.ToAcutalOffset(), increIdxEntry.size, v.Version())
+ needleBytes, err = needle.ReadNeedleBlob(oldDatBackend, increIdxEntry.offset.ToAcutalOffset(), increIdxEntry.size, v.Version())
if err != nil {
return fmt.Errorf("ReadNeedleBlob %s key %d offset %d size %d failed: %v", oldDatFile.Name(), key, increIdxEntry.offset.ToAcutalOffset(), increIdxEntry.size, err)
}
@@ -221,7 +266,7 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
fakeDelNeedle.Id = key
fakeDelNeedle.Cookie = 0x12345678
fakeDelNeedle.AppendAtNs = uint64(time.Now().UnixNano())
- _, _, _, err = fakeDelNeedle.Append(dst, v.Version())
+ _, _, _, err = fakeDelNeedle.Append(dstDatBackend, v.Version())
if err != nil {
return fmt.Errorf("append deleted %d failed: %v", key, err)
}
@@ -241,17 +286,17 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
type VolumeFileScanner4Vacuum struct {
version needle.Version
v *Volume
- dst *os.File
- nm *NeedleMap
+ dstBackend backend.BackendStorageFile
+ nm *needle_map.MemDb
newOffset int64
now uint64
writeThrottler *util.WriteThrottler
}
-func (scanner *VolumeFileScanner4Vacuum) VisitSuperBlock(superBlock SuperBlock) error {
- scanner.version = superBlock.Version()
+func (scanner *VolumeFileScanner4Vacuum) VisitSuperBlock(superBlock super_block.SuperBlock) error {
+ scanner.version = superBlock.Version
superBlock.CompactionRevision++
- _, err := scanner.dst.Write(superBlock.Bytes())
+ _, err := scanner.dstBackend.WriteAt(superBlock.Bytes(), 0)
scanner.newOffset = int64(superBlock.BlockSize())
return err
@@ -260,17 +305,17 @@ func (scanner *VolumeFileScanner4Vacuum) ReadNeedleBody() bool {
return true
}
-func (scanner *VolumeFileScanner4Vacuum) VisitNeedle(n *needle.Needle, offset int64) error {
+func (scanner *VolumeFileScanner4Vacuum) VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error {
if n.HasTtl() && scanner.now >= n.LastModified+uint64(scanner.v.Ttl.Minutes()*60) {
return nil
}
nv, ok := scanner.v.nm.Get(n.Id)
glog.V(4).Infoln("needle expected offset ", offset, "ok", ok, "nv", nv)
if ok && nv.Offset.ToAcutalOffset() == offset && nv.Size > 0 && nv.Size != TombstoneFileSize {
- if err := scanner.nm.Put(n.Id, ToOffset(scanner.newOffset), n.Size); err != nil {
+ if err := scanner.nm.Set(n.Id, ToOffset(scanner.newOffset), n.Size); err != nil {
return fmt.Errorf("cannot put needle: %s", err)
}
- if _, _, _, err := n.Append(scanner.dst, scanner.v.Version()); err != nil {
+ if _, _, _, err := n.Append(scanner.dstBackend, scanner.v.Version()); err != nil {
return fmt.Errorf("cannot append needle: %s", err)
}
delta := n.DiskSize(scanner.version)
@@ -283,53 +328,52 @@ func (scanner *VolumeFileScanner4Vacuum) VisitNeedle(n *needle.Needle, offset in
func (v *Volume) copyDataAndGenerateIndexFile(dstName, idxName string, preallocate int64, compactionBytePerSecond int64) (err error) {
var (
- dst, idx *os.File
+ dst backend.BackendStorageFile
)
- if dst, err = createVolumeFile(dstName, preallocate); err != nil {
+ if dst, err = createVolumeFile(dstName, preallocate, 0); err != nil {
return
}
defer dst.Close()
- if idx, err = os.OpenFile(idxName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644); err != nil {
- return
- }
- defer idx.Close()
+ nm := needle_map.NewMemDb()
scanner := &VolumeFileScanner4Vacuum{
v: v,
now: uint64(time.Now().Unix()),
- nm: NewBtreeNeedleMap(idx),
- dst: dst,
+ nm: nm,
+ dstBackend: dst,
writeThrottler: util.NewWriteThrottler(compactionBytePerSecond),
}
err = ScanVolumeFile(v.dir, v.Collection, v.Id, v.needleMapKind, scanner)
+ if err != nil {
+ return nil
+ }
+
+ err = nm.SaveToIdx(idxName)
return
}
-func (v *Volume) copyDataBasedOnIndexFile(dstName, idxName string) (err error) {
+func (v *Volume) copyDataBasedOnIndexFile(dstName, idxName string, preallocate int64) (err error) {
var (
- dst, idx, oldIndexFile *os.File
+ dstDatBackend backend.BackendStorageFile
+ oldIndexFile *os.File
)
- if dst, err = os.OpenFile(dstName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644); err != nil {
- return
- }
- defer dst.Close()
-
- if idx, err = os.OpenFile(idxName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644); err != nil {
+ if dstDatBackend, err = createVolumeFile(dstName, preallocate, 0); err != nil {
return
}
- defer idx.Close()
+ defer dstDatBackend.Close()
if oldIndexFile, err = os.OpenFile(v.FileName()+".idx", os.O_RDONLY, 0644); err != nil {
return
}
defer oldIndexFile.Close()
- nm := NewBtreeNeedleMap(idx)
+ nm := needle_map.NewMemDb()
+
now := uint64(time.Now().Unix())
v.SuperBlock.CompactionRevision++
- dst.Write(v.SuperBlock.Bytes())
+ dstDatBackend.WriteAt(v.SuperBlock.Bytes(), 0)
newOffset := int64(v.SuperBlock.BlockSize())
idx2.WalkIndexFile(oldIndexFile, func(key NeedleId, offset Offset, size uint32) error {
@@ -343,7 +387,7 @@ func (v *Volume) copyDataBasedOnIndexFile(dstName, idxName string) (err error) {
}
n := new(needle.Needle)
- err := n.ReadData(v.dataFile, offset.ToAcutalOffset(), size, v.Version())
+ err := n.ReadData(v.DataBackend, offset.ToAcutalOffset(), size, v.Version())
if err != nil {
return nil
}
@@ -354,10 +398,10 @@ func (v *Volume) copyDataBasedOnIndexFile(dstName, idxName string) (err error) {
glog.V(4).Infoln("needle expected offset ", offset, "ok", ok, "nv", nv)
if nv.Offset == offset && nv.Size > 0 {
- if err = nm.Put(n.Id, ToOffset(newOffset), n.Size); err != nil {
+ if err = nm.Set(n.Id, ToOffset(newOffset), n.Size); err != nil {
return fmt.Errorf("cannot put needle: %s", err)
}
- if _, _, _, err = n.Append(dst, v.Version()); err != nil {
+ if _, _, _, err = n.Append(dstDatBackend, v.Version()); err != nil {
return fmt.Errorf("cannot append needle: %s", err)
}
newOffset += n.DiskSize(v.Version())
@@ -366,5 +410,7 @@ func (v *Volume) copyDataBasedOnIndexFile(dstName, idxName string) (err error) {
return nil
})
+ nm.SaveToIdx(idxName)
+
return
}
diff --git a/weed/storage/volume_vacuum_test.go b/weed/storage/volume_vacuum_test.go
index 54899c788..95f43d6ec 100644
--- a/weed/storage/volume_vacuum_test.go
+++ b/weed/storage/volume_vacuum_test.go
@@ -8,6 +8,7 @@ import (
"time"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -46,7 +47,7 @@ func TestMakeDiff(t *testing.T) {
v := new(Volume)
//lastCompactIndexOffset value is the index file size before step 4
v.lastCompactIndexOffset = 96
- v.SuperBlock.version = 0x2
+ v.SuperBlock.Version = 0x2
/*
err := v.makeupDiff(
"/yourpath/1.cpd",
@@ -68,7 +69,7 @@ func TestCompaction(t *testing.T) {
}
defer os.RemoveAll(dir) // clean up
- v, err := NewVolume(dir, "", 1, NeedleMapInMemory, &ReplicaPlacement{}, &needle.TTL{}, 0)
+ v, err := NewVolume(dir, "", 1, NeedleMapInMemory, &super_block.ReplicaPlacement{}, &needle.TTL{}, 0, 0)
if err != nil {
t.Fatalf("volume creation: %v", err)
}
@@ -83,7 +84,7 @@ func TestCompaction(t *testing.T) {
}
startTime := time.Now()
- v.Compact(0, 1024*1024)
+ v.Compact2(0)
speed := float64(v.ContentSize()) / time.Now().Sub(startTime).Seconds()
t.Logf("compaction speed: %.2f bytes/s", speed)
@@ -95,7 +96,7 @@ func TestCompaction(t *testing.T) {
v.Close()
- v, err = NewVolume(dir, "", 1, NeedleMapInMemory, nil, nil, 0)
+ v, err = NewVolume(dir, "", 1, NeedleMapInMemory, nil, nil, 0, 0)
if err != nil {
t.Fatalf("volume reloading: %v", err)
}
diff --git a/weed/topology/allocate_volume.go b/weed/topology/allocate_volume.go
index 48336092f..e5dc48652 100644
--- a/weed/topology/allocate_volume.go
+++ b/weed/topology/allocate_volume.go
@@ -18,11 +18,12 @@ func AllocateVolume(dn *DataNode, grpcDialOption grpc.DialOption, vid needle.Vol
return operation.WithVolumeServerClient(dn.Url(), grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.AllocateVolume(context.Background(), &volume_server_pb.AllocateVolumeRequest{
- VolumeId: uint32(vid),
- Collection: option.Collection,
- Replication: option.ReplicaPlacement.String(),
- Ttl: option.Ttl.String(),
- Preallocate: option.Prealloacte,
+ VolumeId: uint32(vid),
+ Collection: option.Collection,
+ Replication: option.ReplicaPlacement.String(),
+ Ttl: option.Ttl.String(),
+ Preallocate: option.Prealloacte,
+ MemoryMapMaxSizeMb: option.MemoryMapMaxSizeMb,
})
return deleteErr
})
diff --git a/weed/topology/collection.go b/weed/topology/collection.go
index f6b728ec9..7a611d904 100644
--- a/weed/topology/collection.go
+++ b/weed/topology/collection.go
@@ -3,8 +3,8 @@ package topology
import (
"fmt"
- "github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -24,7 +24,7 @@ func (c *Collection) String() string {
return fmt.Sprintf("Name:%s, volumeSizeLimit:%d, storageType2VolumeLayout:%v", c.Name, c.volumeSizeLimit, c.storageType2VolumeLayout)
}
-func (c *Collection) GetOrCreateVolumeLayout(rp *storage.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
+func (c *Collection) GetOrCreateVolumeLayout(rp *super_block.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
keyString := rp.String()
if ttl != nil {
keyString += ttl.String()
diff --git a/weed/topology/data_center.go b/weed/topology/data_center.go
index 640cb1937..dc3accb71 100644
--- a/weed/topology/data_center.go
+++ b/weed/topology/data_center.go
@@ -48,6 +48,7 @@ func (dc *DataCenter) ToDataCenterInfo() *master_pb.DataCenterInfo {
MaxVolumeCount: uint64(dc.GetMaxVolumeCount()),
FreeVolumeCount: uint64(dc.FreeSpace()),
ActiveVolumeCount: uint64(dc.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(dc.GetRemoteVolumeCount()),
}
for _, c := range dc.Children() {
rack := c.(*Rack)
diff --git a/weed/topology/data_node.go b/weed/topology/data_node.go
index 3e72ccdbf..617341e54 100644
--- a/weed/topology/data_node.go
+++ b/weed/topology/data_node.go
@@ -2,14 +2,13 @@ package topology
import (
"fmt"
+ "strconv"
"sync"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
- "strconv"
-
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
)
@@ -44,15 +43,26 @@ func (dn *DataNode) String() string {
func (dn *DataNode) AddOrUpdateVolume(v storage.VolumeInfo) (isNew bool) {
dn.Lock()
defer dn.Unlock()
- if _, ok := dn.volumes[v.Id]; !ok {
+ if oldV, ok := dn.volumes[v.Id]; !ok {
dn.volumes[v.Id] = v
dn.UpAdjustVolumeCountDelta(1)
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(1)
+ }
if !v.ReadOnly {
dn.UpAdjustActiveVolumeCountDelta(1)
}
dn.UpAdjustMaxVolumeId(v.Id)
isNew = true
} else {
+ if oldV.IsRemote() != v.IsRemote() {
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(1)
+ }
+ if oldV.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(-1)
+ }
+ }
dn.volumes[v.Id] = v
}
return
@@ -70,7 +80,12 @@ func (dn *DataNode) UpdateVolumes(actualVolumes []storage.VolumeInfo) (newVolume
delete(dn.volumes, vid)
deletedVolumes = append(deletedVolumes, v)
dn.UpAdjustVolumeCountDelta(-1)
- dn.UpAdjustActiveVolumeCountDelta(-1)
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(-1)
+ }
+ if !v.ReadOnly {
+ dn.UpAdjustActiveVolumeCountDelta(-1)
+ }
}
}
dn.Unlock()
@@ -88,7 +103,12 @@ func (dn *DataNode) DeltaUpdateVolumes(newlVolumes, deletedVolumes []storage.Vol
for _, v := range deletedVolumes {
delete(dn.volumes, v.Id)
dn.UpAdjustVolumeCountDelta(-1)
- dn.UpAdjustActiveVolumeCountDelta(-1)
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(-1)
+ }
+ if !v.ReadOnly {
+ dn.UpAdjustActiveVolumeCountDelta(-1)
+ }
}
dn.Unlock()
for _, v := range newlVolumes {
@@ -160,6 +180,7 @@ func (dn *DataNode) ToDataNodeInfo() *master_pb.DataNodeInfo {
MaxVolumeCount: uint64(dn.GetMaxVolumeCount()),
FreeVolumeCount: uint64(dn.FreeSpace()),
ActiveVolumeCount: uint64(dn.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(dn.GetRemoteVolumeCount()),
}
for _, v := range dn.GetVolumes() {
m.VolumeInfos = append(m.VolumeInfos, v.ToVolumeInformationMessage())
diff --git a/weed/topology/node.go b/weed/topology/node.go
index b2808f589..572a89d4d 100644
--- a/weed/topology/node.go
+++ b/weed/topology/node.go
@@ -20,6 +20,7 @@ type Node interface {
ReserveOneVolume(r int64) (*DataNode, error)
UpAdjustMaxVolumeCountDelta(maxVolumeCountDelta int64)
UpAdjustVolumeCountDelta(volumeCountDelta int64)
+ UpAdjustRemoteVolumeCountDelta(remoteVolumeCountDelta int64)
UpAdjustEcShardCountDelta(ecShardCountDelta int64)
UpAdjustActiveVolumeCountDelta(activeVolumeCountDelta int64)
UpAdjustMaxVolumeId(vid needle.VolumeId)
@@ -27,6 +28,7 @@ type Node interface {
GetVolumeCount() int64
GetEcShardCount() int64
GetActiveVolumeCount() int64
+ GetRemoteVolumeCount() int64
GetMaxVolumeCount() int64
GetMaxVolumeId() needle.VolumeId
SetParent(Node)
@@ -44,6 +46,7 @@ type Node interface {
}
type NodeImpl struct {
volumeCount int64
+ remoteVolumeCount int64
activeVolumeCount int64
ecShardCount int64
maxVolumeCount int64
@@ -132,10 +135,11 @@ func (n *NodeImpl) Id() NodeId {
return n.id
}
func (n *NodeImpl) FreeSpace() int64 {
+ freeVolumeSlotCount := n.maxVolumeCount + n.remoteVolumeCount - n.volumeCount
if n.ecShardCount > 0 {
- return n.maxVolumeCount - n.volumeCount - n.ecShardCount/erasure_coding.DataShardsCount - 1
+ freeVolumeSlotCount = freeVolumeSlotCount - n.ecShardCount/erasure_coding.DataShardsCount - 1
}
- return n.maxVolumeCount - n.volumeCount
+ return freeVolumeSlotCount
}
func (n *NodeImpl) SetParent(node Node) {
n.parent = node
@@ -191,6 +195,12 @@ func (n *NodeImpl) UpAdjustVolumeCountDelta(volumeCountDelta int64) { //can be n
n.parent.UpAdjustVolumeCountDelta(volumeCountDelta)
}
}
+func (n *NodeImpl) UpAdjustRemoteVolumeCountDelta(remoteVolumeCountDelta int64) { //can be negative
+ atomic.AddInt64(&n.remoteVolumeCount, remoteVolumeCountDelta)
+ if n.parent != nil {
+ n.parent.UpAdjustRemoteVolumeCountDelta(remoteVolumeCountDelta)
+ }
+}
func (n *NodeImpl) UpAdjustEcShardCountDelta(ecShardCountDelta int64) { //can be negative
atomic.AddInt64(&n.ecShardCount, ecShardCountDelta)
if n.parent != nil {
@@ -220,6 +230,9 @@ func (n *NodeImpl) GetVolumeCount() int64 {
func (n *NodeImpl) GetEcShardCount() int64 {
return n.ecShardCount
}
+func (n *NodeImpl) GetRemoteVolumeCount() int64 {
+ return n.remoteVolumeCount
+}
func (n *NodeImpl) GetActiveVolumeCount() int64 {
return n.activeVolumeCount
}
@@ -235,6 +248,7 @@ func (n *NodeImpl) LinkChildNode(node Node) {
n.UpAdjustMaxVolumeCountDelta(node.GetMaxVolumeCount())
n.UpAdjustMaxVolumeId(node.GetMaxVolumeId())
n.UpAdjustVolumeCountDelta(node.GetVolumeCount())
+ n.UpAdjustRemoteVolumeCountDelta(node.GetRemoteVolumeCount())
n.UpAdjustEcShardCountDelta(node.GetEcShardCount())
n.UpAdjustActiveVolumeCountDelta(node.GetActiveVolumeCount())
node.SetParent(n)
@@ -250,6 +264,7 @@ func (n *NodeImpl) UnlinkChildNode(nodeId NodeId) {
node.SetParent(nil)
delete(n.children, node.Id())
n.UpAdjustVolumeCountDelta(-node.GetVolumeCount())
+ n.UpAdjustRemoteVolumeCountDelta(-node.GetRemoteVolumeCount())
n.UpAdjustEcShardCountDelta(-node.GetEcShardCount())
n.UpAdjustActiveVolumeCountDelta(-node.GetActiveVolumeCount())
n.UpAdjustMaxVolumeCountDelta(-node.GetMaxVolumeCount())
diff --git a/weed/topology/rack.go b/weed/topology/rack.go
index 932c1a804..1921c0c05 100644
--- a/weed/topology/rack.go
+++ b/weed/topology/rack.go
@@ -67,6 +67,7 @@ func (r *Rack) ToRackInfo() *master_pb.RackInfo {
MaxVolumeCount: uint64(r.GetMaxVolumeCount()),
FreeVolumeCount: uint64(r.FreeSpace()),
ActiveVolumeCount: uint64(r.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(r.GetRemoteVolumeCount()),
}
for _, c := range r.Children() {
dn := c.(*DataNode)
diff --git a/weed/topology/store_replicate.go b/weed/topology/store_replicate.go
index d21c4d210..b195b48ed 100644
--- a/weed/topology/store_replicate.go
+++ b/weed/topology/store_replicate.go
@@ -25,58 +25,61 @@ func ReplicatedWrite(masterNode string, s *storage.Store,
//check JWT
jwt := security.GetJwt(r)
+ var remoteLocations []operation.Location
+ if r.FormValue("type") != "replicate" {
+ remoteLocations, err = getWritableRemoteReplications(s, volumeId, masterNode)
+ if err != nil {
+ glog.V(0).Infoln(err)
+ return
+ }
+ }
+
size, isUnchanged, err = s.WriteVolumeNeedle(volumeId, n)
if err != nil {
err = fmt.Errorf("failed to write to local disk: %v", err)
+ glog.V(0).Infoln(err)
return
}
- needToReplicate := !s.HasVolume(volumeId)
- needToReplicate = needToReplicate || s.GetVolume(volumeId).NeedToReplicate()
- if !needToReplicate {
- needToReplicate = s.GetVolume(volumeId).NeedToReplicate()
- }
- if needToReplicate { //send to other replica locations
- if r.FormValue("type") != "replicate" {
-
- if err = distributedOperation(masterNode, s, volumeId, func(location operation.Location) error {
- u := url.URL{
- Scheme: "http",
- Host: location.Url,
- Path: r.URL.Path,
- }
- q := url.Values{
- "type": {"replicate"},
- "ttl": {n.Ttl.String()},
- }
- if n.LastModified > 0 {
- q.Set("ts", strconv.FormatUint(n.LastModified, 10))
- }
- if n.IsChunkedManifest() {
- q.Set("cm", "true")
+ if len(remoteLocations) > 0 { //send to other replica locations
+ if err = distributedOperation(remoteLocations, s, func(location operation.Location) error {
+ u := url.URL{
+ Scheme: "http",
+ Host: location.Url,
+ Path: r.URL.Path,
+ }
+ q := url.Values{
+ "type": {"replicate"},
+ "ttl": {n.Ttl.String()},
+ }
+ if n.LastModified > 0 {
+ q.Set("ts", strconv.FormatUint(n.LastModified, 10))
+ }
+ if n.IsChunkedManifest() {
+ q.Set("cm", "true")
+ }
+ u.RawQuery = q.Encode()
+
+ pairMap := make(map[string]string)
+ if n.HasPairs() {
+ tmpMap := make(map[string]string)
+ err := json.Unmarshal(n.Pairs, &tmpMap)
+ if err != nil {
+ glog.V(0).Infoln("Unmarshal pairs error:", err)
}
- u.RawQuery = q.Encode()
-
- pairMap := make(map[string]string)
- if n.HasPairs() {
- tmpMap := make(map[string]string)
- err := json.Unmarshal(n.Pairs, &tmpMap)
- if err != nil {
- glog.V(0).Infoln("Unmarshal pairs error:", err)
- }
- for k, v := range tmpMap {
- pairMap[needle.PairNamePrefix+k] = v
- }
+ for k, v := range tmpMap {
+ pairMap[needle.PairNamePrefix+k] = v
}
-
- _, err := operation.Upload(u.String(),
- string(n.Name), bytes.NewReader(n.Data), n.IsGzipped(), string(n.Mime),
- pairMap, jwt)
- return err
- }); err != nil {
- size = 0
- err = fmt.Errorf("failed to write to replicas for volume %d: %v", volumeId, err)
}
+
+ _, err := operation.Upload(u.String(),
+ string(n.Name), bytes.NewReader(n.Data), n.IsGzipped(), string(n.Mime),
+ pairMap, jwt)
+ return err
+ }); err != nil {
+ size = 0
+ err = fmt.Errorf("failed to write to replicas for volume %d: %v", volumeId, err)
+ glog.V(0).Infoln(err)
}
}
return
@@ -84,31 +87,34 @@ func ReplicatedWrite(masterNode string, s *storage.Store,
func ReplicatedDelete(masterNode string, store *storage.Store,
volumeId needle.VolumeId, n *needle.Needle,
- r *http.Request) (uint32, error) {
+ r *http.Request) (size uint32, err error) {
//check JWT
jwt := security.GetJwt(r)
- ret, err := store.DeleteVolumeNeedle(volumeId, n)
+ var remoteLocations []operation.Location
+ if r.FormValue("type") != "replicate" {
+ remoteLocations, err = getWritableRemoteReplications(store, volumeId, masterNode)
+ if err != nil {
+ glog.V(0).Infoln(err)
+ return
+ }
+ }
+
+ size, err = store.DeleteVolumeNeedle(volumeId, n)
if err != nil {
glog.V(0).Infoln("delete error:", err)
- return ret, err
+ return
}
- needToReplicate := !store.HasVolume(volumeId)
- if !needToReplicate && ret > 0 {
- needToReplicate = store.GetVolume(volumeId).NeedToReplicate()
- }
- if needToReplicate { //send to other replica locations
- if r.FormValue("type") != "replicate" {
- if err = distributedOperation(masterNode, store, volumeId, func(location operation.Location) error {
- return util.Delete("http://"+location.Url+r.URL.Path+"?type=replicate", string(jwt))
- }); err != nil {
- ret = 0
- }
+ if len(remoteLocations) > 0 { //send to other replica locations
+ if err = distributedOperation(remoteLocations, store, func(location operation.Location) error {
+ return util.Delete("http://"+location.Url+r.URL.Path+"?type=replicate", string(jwt))
+ }); err != nil {
+ size = 0
}
}
- return ret, err
+ return
}
type DistributedOperationResult map[string]error
@@ -131,32 +137,44 @@ type RemoteResult struct {
Error error
}
-func distributedOperation(masterNode string, store *storage.Store, volumeId needle.VolumeId, op func(location operation.Location) error) error {
- if lookupResult, lookupErr := operation.Lookup(masterNode, volumeId.String()); lookupErr == nil {
- length := 0
- selfUrl := (store.Ip + ":" + strconv.Itoa(store.Port))
- results := make(chan RemoteResult)
- for _, location := range lookupResult.Locations {
- if location.Url != selfUrl {
- length++
- go func(location operation.Location, results chan RemoteResult) {
- results <- RemoteResult{location.Url, op(location)}
- }(location, results)
+func distributedOperation(locations []operation.Location, store *storage.Store, op func(location operation.Location) error) error {
+ length := len(locations)
+ results := make(chan RemoteResult)
+ for _, location := range locations {
+ go func(location operation.Location, results chan RemoteResult) {
+ results <- RemoteResult{location.Url, op(location)}
+ }(location, results)
+ }
+ ret := DistributedOperationResult(make(map[string]error))
+ for i := 0; i < length; i++ {
+ result := <-results
+ ret[result.Host] = result.Error
+ }
+
+ return ret.Error()
+}
+
+func getWritableRemoteReplications(s *storage.Store, volumeId needle.VolumeId, masterNode string) (
+ remoteLocations []operation.Location, err error) {
+ copyCount := s.GetVolume(volumeId).ReplicaPlacement.GetCopyCount()
+ if copyCount > 1 {
+ if lookupResult, lookupErr := operation.Lookup(masterNode, volumeId.String()); lookupErr == nil {
+ if len(lookupResult.Locations) < copyCount {
+ err = fmt.Errorf("replicating opetations [%d] is less than volume's replication copy count [%d]",
+ len(lookupResult.Locations), copyCount)
+ return
}
- }
- ret := DistributedOperationResult(make(map[string]error))
- for i := 0; i < length; i++ {
- result := <-results
- ret[result.Host] = result.Error
- }
- if volume := store.GetVolume(volumeId); volume != nil {
- if length+1 < volume.ReplicaPlacement.GetCopyCount() {
- return fmt.Errorf("replicating opetations [%d] is less than volume's replication copy count [%d]", length+1, volume.ReplicaPlacement.GetCopyCount())
+ selfUrl := s.Ip + ":" + strconv.Itoa(s.Port)
+ for _, location := range lookupResult.Locations {
+ if location.Url != selfUrl {
+ remoteLocations = append(remoteLocations, location)
+ }
}
+ } else {
+ err = fmt.Errorf("failed to lookup for %d: %v", volumeId, lookupErr)
+ return
}
- return ret.Error()
- } else {
- glog.V(0).Infoln()
- return fmt.Errorf("Failed to lookup for %d: %v", volumeId, lookupErr)
}
+
+ return
}
diff --git a/weed/topology/topology.go b/weed/topology/topology.go
index aa01190c9..e6cb44727 100644
--- a/weed/topology/topology.go
+++ b/weed/topology/topology.go
@@ -7,15 +7,18 @@ import (
"sync"
"github.com/chrislusf/raft"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/sequence"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/util"
)
type Topology struct {
+ vacuumLockCounter int64
NodeImpl
collectionMap *util.ConcurrentReadMap
@@ -119,16 +122,16 @@ func (t *Topology) HasWritableVolume(option *VolumeGrowOption) bool {
func (t *Topology) PickForWrite(count uint64, option *VolumeGrowOption) (string, uint64, *DataNode, error) {
vid, count, datanodes, err := t.GetVolumeLayout(option.Collection, option.ReplicaPlacement, option.Ttl).PickForWrite(count, option)
if err != nil {
- return "", 0, nil, fmt.Errorf("failed to find writable volumes for collectio:%s replication:%s ttl:%s error: %v", option.Collection, option.ReplicaPlacement.String(), option.Ttl.String(), err)
+ return "", 0, nil, fmt.Errorf("failed to find writable volumes for collection:%s replication:%s ttl:%s error: %v", option.Collection, option.ReplicaPlacement.String(), option.Ttl.String(), err)
}
if datanodes.Length() == 0 {
- return "", 0, nil, fmt.Errorf("no writable volumes available for for collectio:%s replication:%s ttl:%s", option.Collection, option.ReplicaPlacement.String(), option.Ttl.String())
+ return "", 0, nil, fmt.Errorf("no writable volumes available for collection:%s replication:%s ttl:%s", option.Collection, option.ReplicaPlacement.String(), option.Ttl.String())
}
- fileId, count := t.Sequence.NextFileId(count)
+ fileId := t.Sequence.NextFileId(count)
return needle.NewFileId(*vid, fileId, rand.Uint32()).String(), count, datanodes.Head(), nil
}
-func (t *Topology) GetVolumeLayout(collectionName string, rp *storage.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
+func (t *Topology) GetVolumeLayout(collectionName string, rp *super_block.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
return t.collectionMap.Get(collectionName, func() interface{} {
return NewCollection(collectionName, t.volumeSizeLimit)
}).(*Collection).GetOrCreateVolumeLayout(rp, ttl)
diff --git a/weed/topology/topology_event_handling.go b/weed/topology/topology_event_handling.go
index 041351492..068bd401e 100644
--- a/weed/topology/topology_event_handling.go
+++ b/weed/topology/topology_event_handling.go
@@ -59,6 +59,7 @@ func (t *Topology) UnRegisterDataNode(dn *DataNode) {
vl.SetVolumeUnavailable(dn, v.Id)
}
dn.UpAdjustVolumeCountDelta(-dn.GetVolumeCount())
+ dn.UpAdjustRemoteVolumeCountDelta(-dn.GetRemoteVolumeCount())
dn.UpAdjustActiveVolumeCountDelta(-dn.GetActiveVolumeCount())
dn.UpAdjustMaxVolumeCountDelta(-dn.GetMaxVolumeCount())
if dn.Parent() != nil {
diff --git a/weed/topology/topology_map.go b/weed/topology/topology_map.go
index 37a88c9ed..73c55d77d 100644
--- a/weed/topology/topology_map.go
+++ b/weed/topology/topology_map.go
@@ -23,7 +23,7 @@ func (t *Topology) ToMap() interface{} {
}
}
}
- m["layouts"] = layouts
+ m["Layouts"] = layouts
return m
}
@@ -85,6 +85,7 @@ func (t *Topology) ToTopologyInfo() *master_pb.TopologyInfo {
MaxVolumeCount: uint64(t.GetMaxVolumeCount()),
FreeVolumeCount: uint64(t.FreeSpace()),
ActiveVolumeCount: uint64(t.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(t.GetRemoteVolumeCount()),
}
for _, c := range t.Children() {
dc := c.(*DataCenter)
diff --git a/weed/topology/topology_test.go b/weed/topology/topology_test.go
index 8f79ad684..e7676ccf7 100644
--- a/weed/topology/topology_test.go
+++ b/weed/topology/topology_test.go
@@ -5,6 +5,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/sequence"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"testing"
)
@@ -94,7 +95,7 @@ func TestHandlingVolumeServerHeartbeat(t *testing.T) {
[]*master_pb.VolumeShortInformationMessage{newVolumeShortMessage},
nil,
dn)
- rp, _ := storage.NewReplicaPlacementFromString("000")
+ rp, _ := super_block.NewReplicaPlacementFromString("000")
layout := topo.GetVolumeLayout("", rp, needle.EMPTY_TTL)
assert(t, "writables after repeated add", len(layout.writables), volumeCount)
@@ -154,7 +155,7 @@ func TestAddRemoveVolume(t *testing.T) {
DeletedByteCount: 45,
ReadOnly: false,
Version: needle.CurrentVersion,
- ReplicaPlacement: &storage.ReplicaPlacement{},
+ ReplicaPlacement: &super_block.ReplicaPlacement{},
Ttl: needle.EMPTY_TTL,
}
diff --git a/weed/topology/topology_vacuum.go b/weed/topology/topology_vacuum.go
index 351ff842f..ca626e973 100644
--- a/weed/topology/topology_vacuum.go
+++ b/weed/topology/topology_vacuum.go
@@ -2,6 +2,7 @@ package topology
import (
"context"
+ "sync/atomic"
"time"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
@@ -12,8 +13,10 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
)
-func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId, locationlist *VolumeLocationList, garbageThreshold float64) bool {
- ch := make(chan bool, locationlist.Length())
+func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId,
+ locationlist *VolumeLocationList, garbageThreshold float64) (*VolumeLocationList, bool) {
+ ch := make(chan int, locationlist.Length())
+ errCount := int32(0)
for index, dn := range locationlist.list {
go func(index int, url string, vid needle.VolumeId) {
err := operation.WithVolumeServerClient(url, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
@@ -21,11 +24,15 @@ func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vi
VolumeId: uint32(vid),
})
if err != nil {
- ch <- false
+ atomic.AddInt32(&errCount, 1)
+ ch <- -1
return err
}
- isNeeded := resp.GarbageRatio > garbageThreshold
- ch <- isNeeded
+ if resp.GarbageRatio >= garbageThreshold {
+ ch <- index
+ } else {
+ ch <- -1
+ }
return nil
})
if err != nil {
@@ -33,20 +40,25 @@ func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vi
}
}(index, dn.Url(), vid)
}
- isCheckSuccess := true
- for _ = range locationlist.list {
+ vacuumLocationList := NewVolumeLocationList()
+ for range locationlist.list {
select {
- case canVacuum := <-ch:
- isCheckSuccess = isCheckSuccess && canVacuum
+ case index := <-ch:
+ if index != -1 {
+ vacuumLocationList.list = append(vacuumLocationList.list, locationlist.list[index])
+ }
case <-time.After(30 * time.Minute):
- isCheckSuccess = false
- break
+ return vacuumLocationList, false
}
}
- return isCheckSuccess
+ return vacuumLocationList, errCount == 0 && len(vacuumLocationList.list) > 0
}
-func batchVacuumVolumeCompact(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId, locationlist *VolumeLocationList, preallocate int64) bool {
+func batchVacuumVolumeCompact(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId,
+ locationlist *VolumeLocationList, preallocate int64) bool {
+ vl.accessLock.Lock()
vl.removeFromWritable(vid)
+ vl.accessLock.Unlock()
+
ch := make(chan bool, locationlist.Length())
for index, dn := range locationlist.list {
go func(index int, url string, vid needle.VolumeId) {
@@ -67,13 +79,12 @@ func batchVacuumVolumeCompact(grpcDialOption grpc.DialOption, vl *VolumeLayout,
}(index, dn.Url(), vid)
}
isVacuumSuccess := true
- for _ = range locationlist.list {
+ for range locationlist.list {
select {
case canCommit := <-ch:
isVacuumSuccess = isVacuumSuccess && canCommit
case <-time.After(30 * time.Minute):
- isVacuumSuccess = false
- break
+ return false
}
}
return isVacuumSuccess
@@ -118,6 +129,16 @@ func batchVacuumVolumeCleanup(grpcDialOption grpc.DialOption, vl *VolumeLayout,
}
func (t *Topology) Vacuum(grpcDialOption grpc.DialOption, garbageThreshold float64, preallocate int64) int {
+
+ // if there is vacuum going on, return immediately
+ swapped := atomic.CompareAndSwapInt64(&t.vacuumLockCounter, 0, 1)
+ if !swapped {
+ return 0
+ }
+ defer atomic.StoreInt64(&t.vacuumLockCounter, 0)
+
+ // now only one vacuum process going on
+
glog.V(1).Infof("Start vacuum on demand with threshold: %f", garbageThreshold)
for _, col := range t.collectionMap.Items() {
c := col.(*Collection)
@@ -151,11 +172,12 @@ func vacuumOneVolumeLayout(grpcDialOption grpc.DialOption, volumeLayout *VolumeL
}
glog.V(2).Infof("check vacuum on collection:%s volume:%d", c.Name, vid)
- if batchVacuumVolumeCheck(grpcDialOption, volumeLayout, vid, locationList, garbageThreshold) {
- if batchVacuumVolumeCompact(grpcDialOption, volumeLayout, vid, locationList, preallocate) {
- batchVacuumVolumeCommit(grpcDialOption, volumeLayout, vid, locationList)
+ if vacuumLocationList, needVacuum := batchVacuumVolumeCheck(
+ grpcDialOption, volumeLayout, vid, locationList, garbageThreshold); needVacuum {
+ if batchVacuumVolumeCompact(grpcDialOption, volumeLayout, vid, vacuumLocationList, preallocate) {
+ batchVacuumVolumeCommit(grpcDialOption, volumeLayout, vid, vacuumLocationList)
} else {
- batchVacuumVolumeCleanup(grpcDialOption, volumeLayout, vid, locationList)
+ batchVacuumVolumeCleanup(grpcDialOption, volumeLayout, vid, vacuumLocationList)
}
}
}
diff --git a/weed/topology/volume_growth.go b/weed/topology/volume_growth.go
index ff02044a1..80fbc86cd 100644
--- a/weed/topology/volume_growth.go
+++ b/weed/topology/volume_growth.go
@@ -6,6 +6,8 @@ import (
"sync"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
@@ -21,13 +23,14 @@ This package is created to resolve these replica placement issues:
*/
type VolumeGrowOption struct {
- Collection string
- ReplicaPlacement *storage.ReplicaPlacement
- Ttl *needle.TTL
- Prealloacte int64
- DataCenter string
- Rack string
- DataNode string
+ Collection string
+ ReplicaPlacement *super_block.ReplicaPlacement
+ Ttl *needle.TTL
+ Prealloacte int64
+ DataCenter string
+ Rack string
+ DataNode string
+ MemoryMapMaxSizeMb uint32
}
type VolumeGrowth struct {
@@ -58,8 +61,11 @@ func (vg *VolumeGrowth) findVolumeCount(copyCount int) (count int) {
return
}
-func (vg *VolumeGrowth) AutomaticGrowByType(option *VolumeGrowOption, grpcDialOption grpc.DialOption, topo *Topology) (count int, err error) {
- count, err = vg.GrowByCountAndType(grpcDialOption, vg.findVolumeCount(option.ReplicaPlacement.GetCopyCount()), option, topo)
+func (vg *VolumeGrowth) AutomaticGrowByType(option *VolumeGrowOption, grpcDialOption grpc.DialOption, topo *Topology, targetCount int) (count int, err error) {
+ if targetCount == 0 {
+ targetCount = vg.findVolumeCount(option.ReplicaPlacement.GetCopyCount())
+ }
+ count, err = vg.GrowByCountAndType(grpcDialOption, targetCount, option, topo)
if count > 0 && count%option.ReplicaPlacement.GetCopyCount() == 0 {
return count, nil
}
diff --git a/weed/topology/volume_growth_test.go b/weed/topology/volume_growth_test.go
index 3573365fd..e3c5cc580 100644
--- a/weed/topology/volume_growth_test.go
+++ b/weed/topology/volume_growth_test.go
@@ -8,6 +8,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/sequence"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
var topologyLayout = `
@@ -113,7 +114,7 @@ func setup(topologyLayout string) *Topology {
func TestFindEmptySlotsForOneVolume(t *testing.T) {
topo := setup(topologyLayout)
vg := NewDefaultVolumeGrowth()
- rp, _ := storage.NewReplicaPlacementFromString("002")
+ rp, _ := super_block.NewReplicaPlacementFromString("002")
volumeGrowOption := &VolumeGrowOption{
Collection: "",
ReplicaPlacement: rp,
diff --git a/weed/topology/volume_layout.go b/weed/topology/volume_layout.go
index 799cbca62..7633b28be 100644
--- a/weed/topology/volume_layout.go
+++ b/weed/topology/volume_layout.go
@@ -10,11 +10,12 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
// mapping from volume to its locations, inverted from server to volume
type VolumeLayout struct {
- rp *storage.ReplicaPlacement
+ rp *super_block.ReplicaPlacement
ttl *needle.TTL
vid2location map[needle.VolumeId]*VolumeLocationList
writables []needle.VolumeId // transient array of writable volume id
@@ -30,7 +31,7 @@ type VolumeLayoutStats struct {
FileCount uint64
}
-func NewVolumeLayout(rp *storage.ReplicaPlacement, ttl *needle.TTL, volumeSizeLimit uint64) *VolumeLayout {
+func NewVolumeLayout(rp *super_block.ReplicaPlacement, ttl *needle.TTL, volumeSizeLimit uint64) *VolumeLayout {
return &VolumeLayout{
rp: rp,
ttl: ttl,
diff --git a/weed/util/config.go b/weed/util/config.go
index 1ea833d1f..4ba68b800 100644
--- a/weed/util/config.go
+++ b/weed/util/config.go
@@ -9,9 +9,8 @@ type Configuration interface {
GetString(key string) string
GetBool(key string) bool
GetInt(key string) int
- GetInt64(key string) int64
- GetFloat64(key string) float64
GetStringSlice(key string) []string
+ SetDefault(key string, value interface{})
}
func LoadConfiguration(configFileName string, required bool) (loaded bool) {
@@ -40,5 +39,4 @@ func LoadConfiguration(configFileName string, required bool) (loaded bool) {
}
return true
-
}
diff --git a/weed/util/constants.go b/weed/util/constants.go
index 0f29e3388..0916850ef 100644
--- a/weed/util/constants.go
+++ b/weed/util/constants.go
@@ -5,5 +5,5 @@ import (
)
var (
- VERSION = fmt.Sprintf("%s %d.%d", sizeLimit, 1, 41)
+ VERSION = fmt.Sprintf("%s %d.%d", sizeLimit, 1, 47)
)
diff --git a/weed/util/file_util.go b/weed/util/file_util.go
index 78add6724..bef9f7cd6 100644
--- a/weed/util/file_util.go
+++ b/weed/util/file_util.go
@@ -3,6 +3,7 @@ package util
import (
"errors"
"os"
+ "time"
"github.com/chrislusf/seaweedfs/weed/glog"
)
@@ -40,3 +41,21 @@ func FileExists(filename string) bool {
return true
}
+
+func CheckFile(filename string) (exists, canRead, canWrite bool, modTime time.Time, fileSize int64) {
+ exists = true
+ fi, err := os.Stat(filename)
+ if os.IsNotExist(err) {
+ exists = false
+ return
+ }
+ if fi.Mode()&0400 != 0 {
+ canRead = true
+ }
+ if fi.Mode()&0200 != 0 {
+ canWrite = true
+ }
+ modTime = fi.ModTime()
+ fileSize = fi.Size()
+ return
+}
diff --git a/weed/util/http_util.go b/weed/util/http_util.go
index a465a7b7f..667d0b4be 100644
--- a/weed/util/http_util.go
+++ b/weed/util/http_util.go
@@ -11,7 +11,6 @@ import (
"net/http"
"net/url"
"strings"
- "time"
)
var (
@@ -25,12 +24,11 @@ func init() {
}
client = &http.Client{
Transport: Transport,
- Timeout: 5 * time.Second,
}
}
func PostBytes(url string, body []byte) ([]byte, error) {
- r, err := client.Post(url, "application/octet-stream", bytes.NewReader(body))
+ r, err := client.Post(url, "", bytes.NewReader(body))
if err != nil {
return nil, fmt.Errorf("Post to %s: %v", url, err)
}
@@ -189,9 +187,12 @@ func NormalizeUrl(url string) string {
return "http://" + url
}
-func ReadUrl(fileUrl string, offset int64, size int, buf []byte, isReadRange bool) (n int64, e error) {
+func ReadUrl(fileUrl string, offset int64, size int, buf []byte, isReadRange bool) (int64, error) {
- req, _ := http.NewRequest("GET", fileUrl, nil)
+ req, err := http.NewRequest("GET", fileUrl, nil)
+ if err != nil {
+ return 0, err
+ }
if isReadRange {
req.Header.Add("Range", fmt.Sprintf("bytes=%d-%d", offset, offset+int64(size)))
} else {
@@ -217,28 +218,41 @@ func ReadUrl(fileUrl string, offset int64, size int, buf []byte, isReadRange boo
reader = r.Body
}
- var i, m int
+ var (
+ i, m int
+ n int64
+ )
+ // refers to https://github.com/golang/go/blob/master/src/bytes/buffer.go#L199
+ // commit id c170b14c2c1cfb2fd853a37add92a82fd6eb4318
for {
m, err = reader.Read(buf[i:])
- if m == 0 {
- return
- }
i += m
n += int64(m)
if err == io.EOF {
return n, nil
}
- if e != nil {
- return n, e
+ if err != nil {
+ return n, err
+ }
+ if n == int64(len(buf)) {
+ break
}
}
-
+ // drains the response body to avoid memory leak
+ data, _ := ioutil.ReadAll(reader)
+ if len(data) != 0 {
+ err = fmt.Errorf("buffer size is too small. remains %d", len(data))
+ }
+ return n, err
}
-func ReadUrlAsStream(fileUrl string, offset int64, size int, fn func(data []byte)) (n int64, e error) {
+func ReadUrlAsStream(fileUrl string, offset int64, size int, fn func(data []byte)) (int64, error) {
- req, _ := http.NewRequest("GET", fileUrl, nil)
+ req, err := http.NewRequest("GET", fileUrl, nil)
+ if err != nil {
+ return 0, err
+ }
req.Header.Add("Range", fmt.Sprintf("bytes=%d-%d", offset, offset+int64(size)))
r, err := client.Do(req)
@@ -250,21 +264,21 @@ func ReadUrlAsStream(fileUrl string, offset int64, size int, fn func(data []byte
return 0, fmt.Errorf("%s: %s", fileUrl, r.Status)
}
- var m int
+ var (
+ m int
+ n int64
+ )
buf := make([]byte, 64*1024)
for {
m, err = r.Body.Read(buf)
- if m == 0 {
- return
- }
fn(buf[:m])
n += int64(m)
if err == io.EOF {
return n, nil
}
- if e != nil {
- return n, e
+ if err != nil {
+ return n, err
}
}
diff --git a/weed/util/httpdown/http_down.go b/weed/util/httpdown/http_down.go
new file mode 100644
index 000000000..5cbd9611c
--- /dev/null
+++ b/weed/util/httpdown/http_down.go
@@ -0,0 +1,395 @@
+// Package httpdown provides http.ConnState enabled graceful termination of
+// http.Server.
+// based on github.com/facebookarchive/httpdown, who's licence is MIT-licence,
+// we add a feature of supporting for http TLS
+package httpdown
+
+import (
+ "crypto/tls"
+ "fmt"
+ "net"
+ "net/http"
+ "os"
+ "os/signal"
+ "sync"
+ "syscall"
+ "time"
+
+ "github.com/facebookgo/clock"
+ "github.com/facebookgo/stats"
+)
+
+const (
+ defaultStopTimeout = time.Minute
+ defaultKillTimeout = time.Minute
+)
+
+// A Server allows encapsulates the process of accepting new connections and
+// serving them, and gracefully shutting down the listener without dropping
+// active connections.
+type Server interface {
+ // Wait waits for the serving loop to finish. This will happen when Stop is
+ // called, at which point it returns no error, or if there is an error in the
+ // serving loop. You must call Wait after calling Serve or ListenAndServe.
+ Wait() error
+
+ // Stop stops the listener. It will block until all connections have been
+ // closed.
+ Stop() error
+}
+
+// HTTP defines the configuration for serving a http.Server. Multiple calls to
+// Serve or ListenAndServe can be made on the same HTTP instance. The default
+// timeouts of 1 minute each result in a maximum of 2 minutes before a Stop()
+// returns.
+type HTTP struct {
+ // StopTimeout is the duration before we begin force closing connections.
+ // Defaults to 1 minute.
+ StopTimeout time.Duration
+
+ // KillTimeout is the duration before which we completely give up and abort
+ // even though we still have connected clients. This is useful when a large
+ // number of client connections exist and closing them can take a long time.
+ // Note, this is in addition to the StopTimeout. Defaults to 1 minute.
+ KillTimeout time.Duration
+
+ // Stats is optional. If provided, it will be used to record various metrics.
+ Stats stats.Client
+
+ // Clock allows for testing timing related functionality. Do not specify this
+ // in production code.
+ Clock clock.Clock
+
+ // when set CertFile and KeyFile, the httpDown will start a http with TLS.
+ // Files containing a certificate and matching private key for the
+ // server must be provided if neither the Server's
+ // TLSConfig.Certificates nor TLSConfig.GetCertificate are populated.
+ // If the certificate is signed by a certificate authority, the
+ // certFile should be the concatenation of the server's certificate,
+ // any intermediates, and the CA's certificate.
+ CertFile, KeyFile string
+}
+
+// Serve provides the low-level API which is useful if you're creating your own
+// net.Listener.
+func (h HTTP) Serve(s *http.Server, l net.Listener) Server {
+ stopTimeout := h.StopTimeout
+ if stopTimeout == 0 {
+ stopTimeout = defaultStopTimeout
+ }
+ killTimeout := h.KillTimeout
+ if killTimeout == 0 {
+ killTimeout = defaultKillTimeout
+ }
+ klock := h.Clock
+ if klock == nil {
+ klock = clock.New()
+ }
+
+ ss := &server{
+ stopTimeout: stopTimeout,
+ killTimeout: killTimeout,
+ stats: h.Stats,
+ clock: klock,
+ oldConnState: s.ConnState,
+ listener: l,
+ server: s,
+ serveDone: make(chan struct{}),
+ serveErr: make(chan error, 1),
+ new: make(chan net.Conn),
+ active: make(chan net.Conn),
+ idle: make(chan net.Conn),
+ closed: make(chan net.Conn),
+ stop: make(chan chan struct{}),
+ kill: make(chan chan struct{}),
+ certFile: h.CertFile,
+ keyFile: h.KeyFile,
+ }
+ s.ConnState = ss.connState
+ go ss.manage()
+ go ss.serve()
+ return ss
+}
+
+// ListenAndServe returns a Server for the given http.Server. It is equivalent
+// to ListenAndServe from the standard library, but returns immediately.
+// Requests will be accepted in a background goroutine. If the http.Server has
+// a non-nil TLSConfig, a TLS enabled listener will be setup.
+func (h HTTP) ListenAndServe(s *http.Server) (Server, error) {
+ addr := s.Addr
+ if addr == "" {
+ if s.TLSConfig == nil {
+ addr = ":http"
+ } else {
+ addr = ":https"
+ }
+ }
+ l, err := net.Listen("tcp", addr)
+ if err != nil {
+ stats.BumpSum(h.Stats, "listen.error", 1)
+ return nil, err
+ }
+ if s.TLSConfig != nil {
+ l = tls.NewListener(l, s.TLSConfig)
+ }
+ return h.Serve(s, l), nil
+}
+
+// server manages the serving process and allows for gracefully stopping it.
+type server struct {
+ stopTimeout time.Duration
+ killTimeout time.Duration
+ stats stats.Client
+ clock clock.Clock
+
+ oldConnState func(net.Conn, http.ConnState)
+ server *http.Server
+ serveDone chan struct{}
+ serveErr chan error
+ listener net.Listener
+
+ new chan net.Conn
+ active chan net.Conn
+ idle chan net.Conn
+ closed chan net.Conn
+ stop chan chan struct{}
+ kill chan chan struct{}
+
+ stopOnce sync.Once
+ stopErr error
+
+ certFile, keyFile string
+}
+
+func (s *server) connState(c net.Conn, cs http.ConnState) {
+ if s.oldConnState != nil {
+ s.oldConnState(c, cs)
+ }
+
+ switch cs {
+ case http.StateNew:
+ s.new <- c
+ case http.StateActive:
+ s.active <- c
+ case http.StateIdle:
+ s.idle <- c
+ case http.StateHijacked, http.StateClosed:
+ s.closed <- c
+ }
+}
+
+func (s *server) manage() {
+ defer func() {
+ close(s.new)
+ close(s.active)
+ close(s.idle)
+ close(s.closed)
+ close(s.stop)
+ close(s.kill)
+ }()
+
+ var stopDone chan struct{}
+
+ conns := map[net.Conn]http.ConnState{}
+ var countNew, countActive, countIdle float64
+
+ // decConn decrements the count associated with the current state of the
+ // given connection.
+ decConn := func(c net.Conn) {
+ switch conns[c] {
+ default:
+ panic(fmt.Errorf("unknown existing connection: %s", c))
+ case http.StateNew:
+ countNew--
+ case http.StateActive:
+ countActive--
+ case http.StateIdle:
+ countIdle--
+ }
+ }
+
+ // setup a ticker to report various values every minute. if we don't have a
+ // Stats implementation provided, we Stop it so it never ticks.
+ statsTicker := s.clock.Ticker(time.Minute)
+ if s.stats == nil {
+ statsTicker.Stop()
+ }
+
+ for {
+ select {
+ case <-statsTicker.C:
+ // we'll only get here when s.stats is not nil
+ s.stats.BumpAvg("http-state.new", countNew)
+ s.stats.BumpAvg("http-state.active", countActive)
+ s.stats.BumpAvg("http-state.idle", countIdle)
+ s.stats.BumpAvg("http-state.total", countNew+countActive+countIdle)
+ case c := <-s.new:
+ conns[c] = http.StateNew
+ countNew++
+ case c := <-s.active:
+ decConn(c)
+ countActive++
+
+ conns[c] = http.StateActive
+ case c := <-s.idle:
+ decConn(c)
+ countIdle++
+
+ conns[c] = http.StateIdle
+
+ // if we're already stopping, close it
+ if stopDone != nil {
+ c.Close()
+ }
+ case c := <-s.closed:
+ stats.BumpSum(s.stats, "conn.closed", 1)
+ decConn(c)
+ delete(conns, c)
+
+ // if we're waiting to stop and are all empty, we just closed the last
+ // connection and we're done.
+ if stopDone != nil && len(conns) == 0 {
+ close(stopDone)
+ return
+ }
+ case stopDone = <-s.stop:
+ // if we're already all empty, we're already done
+ if len(conns) == 0 {
+ close(stopDone)
+ return
+ }
+
+ // close current idle connections right away
+ for c, cs := range conns {
+ if cs == http.StateIdle {
+ c.Close()
+ }
+ }
+
+ // continue the loop and wait for all the ConnState updates which will
+ // eventually close(stopDone) and return from this goroutine.
+
+ case killDone := <-s.kill:
+ // force close all connections
+ stats.BumpSum(s.stats, "kill.conn.count", float64(len(conns)))
+ for c := range conns {
+ c.Close()
+ }
+
+ // don't block the kill.
+ close(killDone)
+
+ // continue the loop and we wait for all the ConnState updates and will
+ // return from this goroutine when we're all done. otherwise we'll try to
+ // send those ConnState updates on closed channels.
+
+ }
+ }
+}
+
+func (s *server) serve() {
+ stats.BumpSum(s.stats, "serve", 1)
+ if s.certFile == "" && s.keyFile == "" {
+ s.serveErr <- s.server.Serve(s.listener)
+ } else {
+ s.serveErr <- s.server.ServeTLS(s.listener, s.certFile, s.keyFile)
+ }
+ close(s.serveDone)
+ close(s.serveErr)
+}
+
+func (s *server) Wait() error {
+ if err := <-s.serveErr; !isUseOfClosedError(err) {
+ return err
+ }
+ return nil
+}
+
+func (s *server) Stop() error {
+ s.stopOnce.Do(func() {
+ defer stats.BumpTime(s.stats, "stop.time").End()
+ stats.BumpSum(s.stats, "stop", 1)
+
+ // first disable keep-alive for new connections
+ s.server.SetKeepAlivesEnabled(false)
+
+ // then close the listener so new connections can't connect come thru
+ closeErr := s.listener.Close()
+ <-s.serveDone
+
+ // then trigger the background goroutine to stop and wait for it
+ stopDone := make(chan struct{})
+ s.stop <- stopDone
+
+ // wait for stop
+ select {
+ case <-stopDone:
+ case <-s.clock.After(s.stopTimeout):
+ defer stats.BumpTime(s.stats, "kill.time").End()
+ stats.BumpSum(s.stats, "kill", 1)
+
+ // stop timed out, wait for kill
+ killDone := make(chan struct{})
+ s.kill <- killDone
+ select {
+ case <-killDone:
+ case <-s.clock.After(s.killTimeout):
+ // kill timed out, give up
+ stats.BumpSum(s.stats, "kill.timeout", 1)
+ }
+ }
+
+ if closeErr != nil && !isUseOfClosedError(closeErr) {
+ stats.BumpSum(s.stats, "listener.close.error", 1)
+ s.stopErr = closeErr
+ }
+ })
+ return s.stopErr
+}
+
+func isUseOfClosedError(err error) bool {
+ if err == nil {
+ return false
+ }
+ if opErr, ok := err.(*net.OpError); ok {
+ err = opErr.Err
+ }
+ return err.Error() == "use of closed network connection"
+}
+
+// ListenAndServe is a convenience function to serve and wait for a SIGTERM
+// or SIGINT before shutting down.
+func ListenAndServe(s *http.Server, hd *HTTP) error {
+ if hd == nil {
+ hd = &HTTP{}
+ }
+ hs, err := hd.ListenAndServe(s)
+ if err != nil {
+ return err
+ }
+
+ waiterr := make(chan error, 1)
+ go func() {
+ defer close(waiterr)
+ waiterr <- hs.Wait()
+ }()
+
+ signals := make(chan os.Signal, 10)
+ signal.Notify(signals, syscall.SIGTERM, syscall.SIGINT)
+
+ select {
+ case err := <-waiterr:
+ if err != nil {
+ return err
+ }
+ case <-signals:
+ signal.Stop(signals)
+ if err := hs.Stop(); err != nil {
+ return err
+ }
+ if err := <-waiterr; err != nil {
+ return err
+ }
+ }
+ return nil
+}
diff --git a/weed/util/parse.go b/weed/util/parse.go
index 0a8317c19..6593d43b6 100644
--- a/weed/util/parse.go
+++ b/weed/util/parse.go
@@ -1,7 +1,9 @@
package util
import (
+ "net/url"
"strconv"
+ "strings"
)
func ParseInt(text string, defaultValue int) int {
@@ -24,3 +26,22 @@ func ParseUint64(text string, defaultValue uint64) uint64 {
}
return count
}
+
+func ParseFilerUrl(entryPath string) (filerServer string, filerPort int64, path string, err error) {
+ if !strings.HasPrefix(entryPath, "http://") && !strings.HasPrefix(entryPath, "https://") {
+ entryPath = "http://" + entryPath
+ }
+
+ var u *url.URL
+ u, err = url.Parse(entryPath)
+ if err != nil {
+ return
+ }
+ filerServer = u.Hostname()
+ portString := u.Port()
+ if portString != "" {
+ filerPort, err = strconv.ParseInt(portString, 10, 32)
+ }
+ path = u.Path
+ return
+}
diff --git a/weed/util/queue.go b/weed/util/queue.go
new file mode 100644
index 000000000..1e6211e0d
--- /dev/null
+++ b/weed/util/queue.go
@@ -0,0 +1,61 @@
+package util
+
+import "sync"
+
+type node struct {
+ data interface{}
+ next *node
+}
+
+type Queue struct {
+ head *node
+ tail *node
+ count int
+ sync.RWMutex
+}
+
+func NewQueue() *Queue {
+ q := &Queue{}
+ return q
+}
+
+func (q *Queue) Len() int {
+ q.RLock()
+ defer q.RUnlock()
+ return q.count
+}
+
+func (q *Queue) Enqueue(item interface{}) {
+ q.Lock()
+ defer q.Unlock()
+
+ n := &node{data: item}
+
+ if q.tail == nil {
+ q.tail = n
+ q.head = n
+ } else {
+ q.tail.next = n
+ q.tail = n
+ }
+ q.count++
+}
+
+func (q *Queue) Dequeue() interface{} {
+ q.Lock()
+ defer q.Unlock()
+
+ if q.head == nil {
+ return nil
+ }
+
+ n := q.head
+ q.head = n.next
+
+ if q.head == nil {
+ q.tail = nil
+ }
+ q.count--
+
+ return n.data
+}
diff --git a/weed/wdclient/masterclient.go b/weed/wdclient/masterclient.go
index e16356e9d..111514f5e 100644
--- a/weed/wdclient/masterclient.go
+++ b/weed/wdclient/masterclient.go
@@ -51,54 +51,71 @@ func (mc *MasterClient) KeepConnectedToMaster() {
}
func (mc *MasterClient) tryAllMasters() {
+ nextHintedLeader := ""
for _, master := range mc.masters {
- glog.V(1).Infof("%s Connecting to master %v", mc.name, master)
- gprcErr := withMasterClient(context.Background(), master, mc.grpcDialOption, func(ctx context.Context, client master_pb.SeaweedClient) error {
- stream, err := client.KeepConnected(ctx)
+ nextHintedLeader = mc.tryConnectToMaster(master)
+ for nextHintedLeader != "" {
+ nextHintedLeader = mc.tryConnectToMaster(nextHintedLeader)
+ }
+
+ mc.currentMaster = ""
+ mc.vidMap = newVidMap()
+ }
+}
+
+func (mc *MasterClient) tryConnectToMaster(master string) (nextHintedLeader string) {
+ glog.V(1).Infof("%s Connecting to master %v", mc.name, master)
+ gprcErr := withMasterClient(context.Background(), master, mc.grpcDialOption, func(ctx context.Context, client master_pb.SeaweedClient) error {
+
+ stream, err := client.KeepConnected(ctx)
+ if err != nil {
+ glog.V(0).Infof("%s failed to keep connected to %s: %v", mc.name, master, err)
+ return err
+ }
+
+ if err = stream.Send(&master_pb.KeepConnectedRequest{Name: mc.name}); err != nil {
+ glog.V(0).Infof("%s failed to send to %s: %v", mc.name, master, err)
+ return err
+ }
+
+ glog.V(1).Infof("%s Connected to %v", mc.name, master)
+ mc.currentMaster = master
+
+ for {
+ volumeLocation, err := stream.Recv()
if err != nil {
- glog.V(0).Infof("%s failed to keep connected to %s: %v", mc.name, master, err)
+ glog.V(0).Infof("%s failed to receive from %s: %v", mc.name, master, err)
return err
}
- if err = stream.Send(&master_pb.ClientListenRequest{Name: mc.name}); err != nil {
- glog.V(0).Infof("%s failed to send to %s: %v", mc.name, master, err)
- return err
+ // maybe the leader is changed
+ if volumeLocation.Leader != "" {
+ glog.V(0).Infof("redirected to leader %v", volumeLocation.Leader)
+ nextHintedLeader = volumeLocation.Leader
+ return nil
}
- if mc.currentMaster == "" {
- glog.V(1).Infof("%s Connected to %v", mc.name, master)
- mc.currentMaster = master
+ // process new volume location
+ loc := Location{
+ Url: volumeLocation.Url,
+ PublicUrl: volumeLocation.PublicUrl,
}
-
- for {
- if volumeLocation, err := stream.Recv(); err != nil {
- glog.V(0).Infof("%s failed to receive from %s: %v", mc.name, master, err)
- return err
- } else {
- loc := Location{
- Url: volumeLocation.Url,
- PublicUrl: volumeLocation.PublicUrl,
- }
- for _, newVid := range volumeLocation.NewVids {
- glog.V(1).Infof("%s: %s adds volume %d", mc.name, loc.Url, newVid)
- mc.addLocation(newVid, loc)
- }
- for _, deletedVid := range volumeLocation.DeletedVids {
- glog.V(1).Infof("%s: %s removes volume %d", mc.name, loc.Url, deletedVid)
- mc.deleteLocation(deletedVid, loc)
- }
- }
+ for _, newVid := range volumeLocation.NewVids {
+ glog.V(1).Infof("%s: %s adds volume %d", mc.name, loc.Url, newVid)
+ mc.addLocation(newVid, loc)
+ }
+ for _, deletedVid := range volumeLocation.DeletedVids {
+ glog.V(1).Infof("%s: %s removes volume %d", mc.name, loc.Url, deletedVid)
+ mc.deleteLocation(deletedVid, loc)
}
-
- })
-
- if gprcErr != nil {
- glog.V(0).Infof("%s failed to connect with master %v: %v", mc.name, master, gprcErr)
}
- mc.currentMaster = ""
+ })
+ if gprcErr != nil {
+ glog.V(0).Infof("%s failed to connect with master %v: %v", mc.name, master, gprcErr)
}
+ return
}
func withMasterClient(ctx context.Context, master string, grpcDialOption grpc.DialOption, fn func(ctx context.Context, client master_pb.SeaweedClient) error) error {
diff --git a/weed/wdclient/vid_map.go b/weed/wdclient/vid_map.go
index 06308944d..97df49cb6 100644
--- a/weed/wdclient/vid_map.go
+++ b/weed/wdclient/vid_map.go
@@ -3,15 +3,18 @@ package wdclient
import (
"errors"
"fmt"
- "math/rand"
"strconv"
"strings"
"sync"
- "time"
+ "sync/atomic"
"github.com/chrislusf/seaweedfs/weed/glog"
)
+const (
+ maxCursorIndex = 4096
+)
+
type Location struct {
Url string `json:"url,omitempty"`
PublicUrl string `json:"publicUrl,omitempty"`
@@ -20,16 +23,27 @@ type Location struct {
type vidMap struct {
sync.RWMutex
vid2Locations map[uint32][]Location
- r *rand.Rand
+
+ cursor int32
}
func newVidMap() vidMap {
return vidMap{
vid2Locations: make(map[uint32][]Location),
- r: rand.New(rand.NewSource(time.Now().UnixNano())),
+ cursor: -1,
}
}
+func (vc *vidMap) getLocationIndex(length int) (int, error) {
+ if length <= 0 {
+ return 0, fmt.Errorf("invalid length: %d", length)
+ }
+ if atomic.LoadInt32(&vc.cursor) == maxCursorIndex {
+ atomic.CompareAndSwapInt32(&vc.cursor, maxCursorIndex, -1)
+ }
+ return int(atomic.AddInt32(&vc.cursor, 1)) % length, nil
+}
+
func (vc *vidMap) LookupVolumeServerUrl(vid string) (serverUrl string, err error) {
id, err := strconv.Atoi(vid)
if err != nil {
@@ -64,20 +78,25 @@ func (vc *vidMap) LookupVolumeServer(fileId string) (volumeServer string, err er
return serverUrl, nil
}
-func (vc *vidMap) GetVidLocations(vid string) (locations []Location) {
+func (vc *vidMap) GetVidLocations(vid string) (locations []Location, err error) {
id, err := strconv.Atoi(vid)
if err != nil {
glog.V(1).Infof("Unknown volume id %s", vid)
- return nil
+ return nil, fmt.Errorf("Unknown volume id %s", vid)
+ }
+ foundLocations, found := vc.GetLocations(uint32(id))
+ if found {
+ return foundLocations, nil
}
- return vc.GetLocations(uint32(id))
+ return nil, fmt.Errorf("volume id %s not found", vid)
}
-func (vc *vidMap) GetLocations(vid uint32) (locations []Location) {
+func (vc *vidMap) GetLocations(vid uint32) (locations []Location, found bool) {
vc.RLock()
defer vc.RUnlock()
- return vc.vid2Locations[vid]
+ locations, found = vc.vid2Locations[vid]
+ return
}
func (vc *vidMap) GetRandomLocation(vid uint32) (serverUrl string, err error) {
@@ -89,7 +108,12 @@ func (vc *vidMap) GetRandomLocation(vid uint32) (serverUrl string, err error) {
return "", fmt.Errorf("volume %d not found", vid)
}
- return locations[vc.r.Intn(len(locations))].Url, nil
+ index, err := vc.getLocationIndex(len(locations))
+ if err != nil {
+ return "", fmt.Errorf("volume %d: %v", vid, err)
+ }
+
+ return locations[index].Url, nil
}
func (vc *vidMap) addLocation(vid uint32, location Location) {
diff --git a/weed/wdclient/vid_map_test.go b/weed/wdclient/vid_map_test.go
new file mode 100644
index 000000000..87be2fc25
--- /dev/null
+++ b/weed/wdclient/vid_map_test.go
@@ -0,0 +1,76 @@
+package wdclient
+
+import (
+ "fmt"
+ "testing"
+)
+
+func TestLocationIndex(t *testing.T) {
+ vm := vidMap{}
+ // test must be failed
+ mustFailed := func(length int) {
+ _, err := vm.getLocationIndex(length)
+ if err == nil {
+ t.Errorf("length %d must be failed", length)
+ }
+ if err.Error() != fmt.Sprintf("invalid length: %d", length) {
+ t.Errorf("length %d must be failed. error: %v", length, err)
+ }
+ }
+
+ mustFailed(-1)
+ mustFailed(0)
+
+ mustOk := func(length, cursor, expect int) {
+ if length <= 0 {
+ t.Fatal("please don't do this")
+ }
+ vm.cursor = int32(cursor)
+ got, err := vm.getLocationIndex(length)
+ if err != nil {
+ t.Errorf("length: %d, why? %v\n", length, err)
+ return
+ }
+ if got != expect {
+ t.Errorf("cursor: %d, length: %d, expect: %d, got: %d\n", cursor, length, expect, got)
+ return
+ }
+ }
+
+ for i := -1; i < 100; i++ {
+ mustOk(7, i, (i+1)%7)
+ }
+
+ // when cursor reaches MaxInt64
+ mustOk(7, maxCursorIndex, 0)
+
+ // test with constructor
+ vm = newVidMap()
+ length := 7
+ for i := 0; i < 100; i++ {
+ got, err := vm.getLocationIndex(length)
+ if err != nil {
+ t.Errorf("length: %d, why? %v\n", length, err)
+ return
+ }
+ if got != i%length {
+ t.Errorf("length: %d, i: %d, got: %d\n", length, i, got)
+ }
+ }
+}
+
+func BenchmarkLocationIndex(b *testing.B) {
+ b.SetParallelism(8)
+ vm := vidMap{
+ cursor: maxCursorIndex - 4000,
+ }
+ b.ResetTimer()
+ b.RunParallel(func(pb *testing.PB) {
+ for pb.Next() {
+ _, err := vm.getLocationIndex(3)
+ if err != nil {
+ b.Error(err)
+ }
+ }
+ })
+}