diff --git a/suro-client/src/main/java/com/netflix/suro/client/example/SuroClient4Test.java b/suro-client/src/main/java/com/netflix/suro/client/example/SuroClient4Test.java index 16505833..856a0d3c 100644 --- a/suro-client/src/main/java/com/netflix/suro/client/example/SuroClient4Test.java +++ b/suro-client/src/main/java/com/netflix/suro/client/example/SuroClient4Test.java @@ -27,15 +27,25 @@ import java.util.Properties; public class SuroClient4Test { - public static void main(String[] args) throws JsonProcessingException { + public static void main(String[] args) throws JsonProcessingException, InterruptedException { + // ip num_of_messages message_size sleep num_of_iterations + String ip = args[0]; + int numMessages = Integer.parseInt(args[1]); + int messageSize = Integer.parseInt(args[2]); + int sleep = Integer.parseInt(args[3]); + int numIterations = Integer.parseInt(args[4]); + Properties props = new Properties(); props.setProperty(ClientConfig.LB_TYPE, "static"); - props.setProperty(ClientConfig.LB_SERVER, args[0]); + props.setProperty(ClientConfig.LB_SERVER, ip); SuroClient client = new SuroClient(props); - byte[] payload = createMessagePayload(Integer.parseInt(args[2])); - for (int i = 0; i < Integer.parseInt(args[1]); ++i) { - client.send(new Message(i % 2 == 0 ? "request_trace" : "nf_errors_log", payload)); + byte[] payload = createMessagePayload(messageSize); + for (int n = 0; n < numIterations; ++n) { + for (int i = 0; i < numMessages; ++i) { + client.send(new Message(i % 2 == 0 ? "request_trace" : "nf_errors_log", payload)); + } + Thread.sleep(sleep); } client.shutdown(); } diff --git a/suro-server/src/main/java/com/netflix/suro/sink/ServerSinkPlugin.java b/suro-server/src/main/java/com/netflix/suro/sink/ServerSinkPlugin.java index c7b778f2..a1392c46 100644 --- a/suro-server/src/main/java/com/netflix/suro/sink/ServerSinkPlugin.java +++ b/suro-server/src/main/java/com/netflix/suro/sink/ServerSinkPlugin.java @@ -3,9 +3,11 @@ import com.netflix.suro.SuroPlugin; import com.netflix.suro.sink.kafka.KafkaSink; import com.netflix.suro.sink.localfile.LocalFileSink; +import com.netflix.suro.sink.notice.LogNotice; import com.netflix.suro.sink.notice.NoNotice; import com.netflix.suro.sink.notice.QueueNotice; import com.netflix.suro.sink.notice.SQSNotice; +import com.netflix.suro.sink.remotefile.HdfsFileSink; import com.netflix.suro.sink.remotefile.S3FileSink; import com.netflix.suro.sink.remotefile.formatter.DateRegionStackFormatter; import com.netflix.suro.sink.remotefile.formatter.SimpleDateFormatter; @@ -23,6 +25,7 @@ protected void configure() { this.addSinkType(KafkaSink.TYPE, KafkaSink.class); this.addSinkType(S3FileSink.TYPE, S3FileSink.class); + this.addSinkType(HdfsFileSink.TYPE, HdfsFileSink.class); this.addRemotePrefixFormatterType(DateRegionStackFormatter.TYPE, DateRegionStackFormatter.class); this.addRemotePrefixFormatterType(SimpleDateFormatter.TYPE, SimpleDateFormatter.class); this.addRemotePrefixFormatterType(StaticPrefixFormatter.TYPE, StaticPrefixFormatter.class); @@ -32,5 +35,6 @@ protected void configure() { this.addNoticeType(NoNotice.TYPE, NoNotice.class); this.addNoticeType(QueueNotice.TYPE, QueueNotice.class); this.addNoticeType(SQSNotice.TYPE, SQSNotice.class); + this.addNoticeType(LogNotice.TYPE, LogNotice.class); } } diff --git a/suro-server/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java b/suro-server/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java index c780a3ab..7582fda7 100644 --- a/suro-server/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java +++ b/suro-server/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java @@ -360,11 +360,15 @@ public String getStat() { */ public void deleteFile(String filePath) { int retryCount = 1; - while (new File(filePath).exists() && retryCount <= deleteFileRetryCount) { + while (retryCount <= deleteFileRetryCount) { try { - Thread.sleep(1000 * retryCount); - writer.getFS().delete(new Path(filePath), false); - ++retryCount; + if (writer.getFS().exists(new Path(filePath))) { + Thread.sleep(1000 * retryCount); + writer.getFS().delete(new Path(filePath), false); + ++retryCount; + } else { + break; + } } catch (Exception e) { log.warn("Exception while deleting the file: " + e.getMessage(), e); } diff --git a/suro-server/src/main/java/com/netflix/suro/sink/notice/LogNotice.java b/suro-server/src/main/java/com/netflix/suro/sink/notice/LogNotice.java new file mode 100644 index 00000000..064cf1a0 --- /dev/null +++ b/suro-server/src/main/java/com/netflix/suro/sink/notice/LogNotice.java @@ -0,0 +1,32 @@ +package com.netflix.suro.sink.notice; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LogNotice implements Notice { + public static final String TYPE = "log"; + + private static Logger log = LoggerFactory.getLogger(LogNotice.class); + + @Override + public void init() { + + } + + @Override + public boolean send(String message) { + log.info(message); + + return true; + } + + @Override + public String recv() { + return null; + } + + @Override + public String getStat() { + return null; + } +} diff --git a/suro-server/src/main/java/com/netflix/suro/sink/remotefile/HdfsFileSink.java b/suro-server/src/main/java/com/netflix/suro/sink/remotefile/HdfsFileSink.java new file mode 100644 index 00000000..6289fc76 --- /dev/null +++ b/suro-server/src/main/java/com/netflix/suro/sink/remotefile/HdfsFileSink.java @@ -0,0 +1,78 @@ +package com.netflix.suro.sink.remotefile; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.netflix.suro.sink.localfile.FileNameFormatter; +import com.netflix.suro.sink.localfile.LocalFileSink; +import com.netflix.suro.sink.notice.Notice; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.codehaus.jettison.json.JSONObject; + +import java.util.Properties; + +public class HdfsFileSink extends RemoteFileSink { + public static final String TYPE = "hdfs"; + + private final String directory; + private final Notice notice; + private final Configuration hadoopConf; + + @JsonCreator + public HdfsFileSink( + @JsonProperty("localFileSink") LocalFileSink localFileSink, + @JsonProperty("directory") String directory, + @JsonProperty("concurrentUpload") int concurrentUpload, + @JsonProperty("notice") Notice notice, + @JsonProperty("prefixFormatter") RemotePrefixFormatter prefixFormatter, + @JsonProperty("batchUpload") boolean batchUpload, + @JsonProperty("properties") Properties properties + ) { + super(localFileSink, prefixFormatter, concurrentUpload, batchUpload); + + this.directory = directory; + this.notice = notice; + hadoopConf = new Configuration(); + if (properties != null) { + for (String propertyName : properties.stringPropertyNames()) { + hadoopConf.set(propertyName, properties.getProperty(propertyName)); + } + } + + Preconditions.checkNotNull(directory, "directory is needed"); + } + + @Override + public String recvNotice() { + return notice.recv(); + } + + @Override + void initialize() { + // do nothing + } + + @Override + void upload(String localFilePath, String remoteFilePath) throws Exception { + Path outFile = new Path(String.format("%s/%s", directory, remoteFilePath)); + FileSystem fs = outFile.getFileSystem(hadoopConf); + + fs.mkdirs(outFile.getParent()); + fs.moveFromLocalFile(new Path(localFilePath), outFile); + } + + @Override + void notify(String filePath, long fileSize) throws Exception { + JSONObject jsonMessage = new JSONObject(); + jsonMessage.put("directory", directory); + jsonMessage.put("filePath", filePath); + jsonMessage.put("size", fileSize); + jsonMessage.put("collector", FileNameFormatter.localHostAddr); + + if (!notice.send(jsonMessage.toString())) { + throw new RuntimeException("Notice failed"); + } + } +} diff --git a/suro-server/src/main/java/com/netflix/suro/sink/remotefile/RemoteFileSink.java b/suro-server/src/main/java/com/netflix/suro/sink/remotefile/RemoteFileSink.java new file mode 100644 index 00000000..2f59e718 --- /dev/null +++ b/suro-server/src/main/java/com/netflix/suro/sink/remotefile/RemoteFileSink.java @@ -0,0 +1,240 @@ +package com.netflix.suro.sink.remotefile; + +import com.google.common.base.Preconditions; +import com.netflix.servo.annotations.DataSourceType; +import com.netflix.servo.annotations.Monitor; +import com.netflix.servo.monitor.Monitors; +import com.netflix.suro.message.MessageContainer; +import com.netflix.suro.sink.Sink; +import com.netflix.suro.sink.localfile.LocalFileSink; +import com.netflix.suro.sink.notice.Notice; +import com.netflix.suro.sink.notice.QueueNotice; +import com.netflix.suro.sink.remotefile.formatter.SimpleDateFormatter; +import org.joda.time.Period; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public abstract class RemoteFileSink implements Sink { + private static final Logger log = LoggerFactory.getLogger(RemoteFileSink.class); + + private final LocalFileSink localFileSink; + private final RemotePrefixFormatter prefixFormatter; + + private final ExecutorService uploader; + private final ExecutorService localFilePoller; + private ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + + private final boolean batchUpload; + + private boolean running = false; + private static final int processingFileQueueThreshold = 1000; + private static final String processingFileQueueCleanupInterval = "PT60s"; + + private Set processingFileSet = Collections.newSetFromMap(new ConcurrentHashMap()); + private BlockingQueue processedFileQueue = new LinkedBlockingQueue(); + + public RemoteFileSink( + LocalFileSink localFileSink, + RemotePrefixFormatter prefixFormatter, + int concurrentUpload, + boolean batchUpload) { + this.localFileSink = localFileSink; + this.prefixFormatter = prefixFormatter == null ? new SimpleDateFormatter("yyyyMMdd") : prefixFormatter; + this.batchUpload = batchUpload; + + Preconditions.checkNotNull(localFileSink, "localFileSink is needed"); + + uploader = Executors.newFixedThreadPool(concurrentUpload == 0 ? 5 : concurrentUpload); + localFilePoller = Executors.newSingleThreadExecutor(); + + if (!batchUpload) { + localFileSink.cleanUp(); + } + + Monitors.registerObject( + this.getClass().getSimpleName() + '-' + localFileSink.getOutputDir().replace('/', '_'), + this); + } + + @Override + public void writeTo(MessageContainer message) { + localFileSink.writeTo(message); + } + + @Override + public void open() { + initialize(); + + if (!batchUpload) { + running = true; + + localFilePoller.submit(new Runnable() { + @Override + public void run() { + while (running) { + uploadAllFromQueue(); + localFileSink.cleanUp(); + } + uploadAllFromQueue(); + } + }); + localFileSink.open(); + + int schedulingSecond = new Period(processingFileQueueCleanupInterval).toStandardSeconds().getSeconds(); + scheduler.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + if (processingFileSet.size() > processingFileQueueThreshold) { + String file = null; + int count = 0; + while (processingFileSet.size() > processingFileQueueThreshold && + (file = processedFileQueue.poll()) != null) { + processingFileSet.remove(file); + ++count; + } + log.info(count + " files are removed from processingFileSet"); + } + } + }, schedulingSecond, schedulingSecond, TimeUnit.SECONDS); + } + } + + @Override + public void close() { + try { + if (!batchUpload) { + localFileSink.close(); + running = false; + localFilePoller.shutdown(); + + localFilePoller.awaitTermination(60000, TimeUnit.MILLISECONDS); + } + uploader.shutdown(); + uploader.awaitTermination(60000, TimeUnit.MILLISECONDS); + } catch (Exception e) { + // ignore exceptions while closing + log.error("Exception while closing: " + e.getMessage(), e); + } + } + + @Override + public String getStat() { + StringBuilder sb = new StringBuilder(localFileSink.getStat()); + sb.append('\n').append(String.format("%d files uploaded so far", uploadedFileCount.get())); + + return sb.toString(); + } + + public void uploadAll(String dir) { + clearFileHistory(); + + while (localFileSink.cleanUp(dir) > 0) { + uploadAllFromQueue(); + } + } + + private void clearFileHistory() { + processedFileQueue.clear(); + processingFileSet.clear(); + } + + private void uploadAllFromQueue() { + String note = localFileSink.recvNotice(); + while (note != null) { + uploadFile(note); + note = localFileSink.recvNotice(); + } + } + + private void uploadFile(final String filePath) { + // to prevent multiple uploading in any situations + final String key = filePath.substring(filePath.lastIndexOf("/")); + if (processingFileSet.contains(key)) { + return; + } + processingFileSet.add(key); + + uploader.submit(new Runnable() { + @Override + public void run() { + try { + File localFile = new File(filePath); + long fileLength = localFile.length(); + if (fileLength == 0) { + log.warn("empty file: " + filePath + " is abandoned"); + localFileSink.deleteFile(filePath); + return; + } + String remoteFilePath = makeUploadPath(localFile); + + long t1 = System.currentTimeMillis(); + + upload(filePath, remoteFilePath); + + long t2 = System.currentTimeMillis(); + + log.info("upload duration: " + (t2 - t1) + " ms " + + "for " + filePath + " Len: " + fileLength + " bytes"); + + uploadedFileSize.addAndGet(fileLength); + uploadedFileCount.incrementAndGet(); + uploadDuration = t2 - t1; + + RemoteFileSink.this.notify(remoteFilePath, fileLength); + localFileSink.deleteFile(filePath); + + log.info("upload done deleting from local: " + filePath); + } catch (Exception e) { + uploadFailureCount.incrementAndGet(); + log.error("Exception while uploading: " + e.getMessage(), e); + } finally { + // check the file was deleted or not + if (new File(filePath).exists()) { + // something error happened + // it should be done again + processingFileSet.remove(key); + } else { + processedFileQueue.add(key); + } + } + } + }); + } + + private String makeUploadPath(File file) { + return prefixFormatter.get() + file.getName(); + } + + @Monitor(name = "uploadedFileSize", type = DataSourceType.COUNTER) + public long getUploadedFileSize() { + return uploadedFileSize.get(); + } + + @Monitor(name = "uploadDuration", type = DataSourceType.GAUGE) + private long uploadDuration; + + @Monitor(name = "uploadedFileCount", type = DataSourceType.COUNTER) + public int getUploadedFileCount() { + return uploadedFileCount.get(); + } + + @Monitor(name = "uploadFailureCount", type=DataSourceType.COUNTER) + public int getUploadFailureCount() { + return uploadFailureCount.get(); + } + + private AtomicLong uploadedFileSize = new AtomicLong(0); + private AtomicInteger uploadedFileCount = new AtomicInteger(0); + private AtomicInteger uploadFailureCount = new AtomicInteger(0); + + abstract void initialize(); + abstract void upload(String localFilePath, String remoteFilePath) throws Exception; + abstract void notify(String filePath, long fileSize) throws Exception; +} diff --git a/suro-server/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java b/suro-server/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java index 83f2a13b..bfc4d030 100644 --- a/suro-server/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java +++ b/suro-server/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.netflix.servo.annotations.DataSourceType; import com.netflix.servo.annotations.Monitor; @@ -60,35 +61,28 @@ * * @author jbae */ -public class S3FileSink implements Sink { +public class S3FileSink extends RemoteFileSink { public static final String TYPE = "s3"; - static Logger log = LoggerFactory.getLogger(S3FileSink.class); - - private final LocalFileSink localFileSink; - private final boolean batchUpload; + private static Logger log = LoggerFactory.getLogger(S3FileSink.class); private final String bucket; private final String s3Endpoint; private final long maxPartSize; private final Notice notice; - private final RemotePrefixFormatter prefixFormatter; private MultipartUtils mpUtils; private AWSCredentialsProvider credentialsProvider; private RestS3Service s3Service; - private GrantAcl grantAcl; - private final ExecutorService uploader; - private final ExecutorService localFilePoller; + @VisibleForTesting + protected GrantAcl grantAcl; private final String s3Acl; private final int s3AclRetries; - private ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); - @JsonCreator public S3FileSink( @JsonProperty("localFileSink") LocalFileSink localFileSink, @@ -103,54 +97,23 @@ public S3FileSink( @JsonProperty("s3AclRetries") int s3AclRetries, @JacksonInject("multipartUtils") MultipartUtils mpUtils, @JacksonInject("credentials") AWSCredentialsProvider credentialProvider) { - this.localFileSink = localFileSink; + super(localFileSink, prefixFormatter, concurrentUpload, batchUpload); + this.bucket = bucket; this.s3Endpoint = s3Endpoint == null ? "s3.amazonaws.com" : s3Endpoint; this.maxPartSize = maxPartSize == 0 ? 20 * 1024 * 1024 : maxPartSize; this.notice = notice == null ? new QueueNotice() : notice; - this.prefixFormatter = prefixFormatter == null ? new SimpleDateFormatter("'P'yyyyMMdd'T'HHmmss") : prefixFormatter; - this.batchUpload = batchUpload; this.mpUtils = mpUtils; this.credentialsProvider = credentialProvider; - if (concurrentUpload == 0) { - concurrentUpload = 5; - } - uploader = Executors.newFixedThreadPool(concurrentUpload); - localFilePoller = Executors.newSingleThreadExecutor(); - this.s3Acl = s3Acl; this.s3AclRetries = s3AclRetries > 0 ? s3AclRetries : 5; - Preconditions.checkNotNull(localFileSink, "localFileSink is needed"); Preconditions.checkNotNull(bucket, "bucket is needed"); - - if (!batchUpload) { - localFileSink.cleanUp(); - } - - Monitors.registerObject( - S3FileSink.class.getSimpleName() + '-' + localFileSink.getOutputDir().replace('/', '_'), - this); } - // testing purpose only - public void setGrantAcl(GrantAcl grantAcl) { - this.grantAcl = grantAcl; - } - - @Override - public void writeTo(MessageContainer message) { - localFileSink.writeTo(message); - } - - private boolean running = false; - private static final int processingFileQueueThreshold = 1000; - private static final String processingFileQueueCleanupInterval = "PT60s"; - - @Override - public void open() { + protected void initialize() { if (mpUtils == null) { // not injected mpUtils = new MultipartUtils(maxPartSize); } @@ -175,70 +138,6 @@ public void open() { grantAcl = new GrantAcl(s3Service, s3Acl, s3AclRetries == 0 ? 5 : s3AclRetries); notice.init(); - - if (!batchUpload) { - running = true; - - localFilePoller.submit(new Runnable() { - @Override - public void run() { - while (running) { - uploadAllFromQueue(); - localFileSink.cleanUp(); - } - uploadAllFromQueue(); - } - }); - localFileSink.open(); - - int schedulingSecond = new Period(processingFileQueueCleanupInterval).toStandardSeconds().getSeconds(); - scheduler.scheduleAtFixedRate(new Runnable() { - @Override - public void run() { - if (processingFileSet.size() > processingFileQueueThreshold) { - String file = null; - int count = 0; - while (processingFileSet.size() > processingFileQueueThreshold && - (file = processedFileQueue.poll()) != null) { - processingFileSet.remove(file); - ++count; - } - log.info(count + " files are removed from processingFileSet"); - } - } - }, schedulingSecond, schedulingSecond, TimeUnit.SECONDS); - } - } - - private void clearFileHistory() { - processedFileQueue.clear(); - processingFileSet.clear(); - } - - private void uploadAllFromQueue() { - String note = localFileSink.recvNotice(); - while (note != null) { - uploadFile(note); - note = localFileSink.recvNotice(); - } - } - - @Override - public void close() { - try { - if (!batchUpload) { - localFileSink.close(); - running = false; - localFilePoller.shutdown(); - - localFilePoller.awaitTermination(60000, TimeUnit.MILLISECONDS); - } - uploader.shutdown(); - uploader.awaitTermination(60000, TimeUnit.MILLISECONDS); - } catch (Exception e) { - // ignore exceptions while closing - log.error("Exception while closing: " + e.getMessage(), e); - } } @Override @@ -246,107 +145,12 @@ public String recvNotice() { return notice.recv(); } - @Override - public String getStat() { - StringBuilder sb = new StringBuilder(localFileSink.getStat()); - sb.append('\n').append(String.format("%d files uploaded so far", uploadedFileCount.get())); - - return sb.toString(); - } - - @Monitor(name = "uploadedFileSize", type = DataSourceType.COUNTER) - public long getUploadedFileSize() { - return uploadedFileSize.get(); - } - - @Monitor(name = "uploadDuration", type = DataSourceType.GAUGE) - private long uploadDuration; - - @Monitor(name = "uploadedFileCount", type = DataSourceType.COUNTER) - public int getUploadedFileCount() { - return uploadedFileCount.get(); - } - - @Monitor(name = "uploadFailureCount", type=DataSourceType.COUNTER) - public int getUploadFailureCount() { - return uploadFailureCount.get(); - } - - private AtomicLong uploadedFileSize = new AtomicLong(0); - private AtomicInteger uploadedFileCount = new AtomicInteger(0); - private AtomicInteger uploadFailureCount = new AtomicInteger(0); - @Monitor(name="fail_grantAcl", type=DataSourceType.COUNTER) private AtomicLong fail_grantAcl = new AtomicLong(0); public long getFail_grantAcl() { return fail_grantAcl.get(); } - private Set processingFileSet = Collections.newSetFromMap(new ConcurrentHashMap()); - private BlockingQueue processedFileQueue = new LinkedBlockingQueue(); - - private void uploadFile(final String filePath) { - // to prevent multiple uploading in any situations - final String key = filePath.substring(filePath.lastIndexOf("/")); - if (processingFileSet.contains(key)) { - return; - } - processingFileSet.add(key); - - uploader.submit(new Runnable() { - @Override - public void run() { - try { - File localFile = new File(filePath); - if (localFile.length() == 0) { - log.warn("empty file: " + filePath + " is abandoned"); - localFileSink.deleteFile(filePath); - return; - } - String remoteFilePath = makeUploadPath(localFile); - - long t1 = System.currentTimeMillis(); - S3Object file = new S3Object(new File(filePath)); - file.setBucketName(bucket); - file.setKey(remoteFilePath); - file.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); - List objectsToUploadAsMultipart = new ArrayList(); - objectsToUploadAsMultipart.add(file); - mpUtils.uploadObjects(bucket, s3Service, objectsToUploadAsMultipart, null); - - if (!grantAcl.grantAcl(file)) { - throw new RuntimeException("Failed to set Acl"); - } - - long t2 = System.currentTimeMillis(); - - log.info("upload duration: " + (t2 - t1) + " ms " + - "for " + filePath + " Len: " + localFile.length() + " bytes"); - - uploadedFileSize.addAndGet(localFile.length()); - uploadedFileCount.incrementAndGet(); - uploadDuration = t2 - t1; - - S3FileSink.this.notify(remoteFilePath, localFile.length()); - localFileSink.deleteFile(filePath); - - log.info("upload done deleting from local: " + filePath); - } catch (Exception e) { - uploadFailureCount.incrementAndGet(); - log.error("Exception while uploading: " + e.getMessage(), e); - } finally { - // check the file was deleted or not - if (new File(filePath).exists()) { - // something error happened - // it should be done again - processingFileSet.remove(key); - } else { - processedFileQueue.add(key); - } - } - } - }); - } - - private void notify(String filePath, long fileSize) throws JSONException { + @Override + protected void notify(String filePath, long fileSize) throws Exception { JSONObject jsonMessage = new JSONObject(); jsonMessage.put("bucket", bucket); jsonMessage.put("filePath", filePath); @@ -358,20 +162,18 @@ private void notify(String filePath, long fileSize) throws JSONException { } } - private String makeUploadPath(File file) { - return prefixFormatter.get() + file.getName(); - } - - public void uploadAll(String dir) { - clearFileHistory(); - - while (localFileSink.cleanUp(dir) > 0) { - uploadAllFromQueue(); + @Override + protected void upload(String localFilePath, String remoteFilePath) throws Exception { + S3Object file = new S3Object(new File(localFilePath)); + file.setBucketName(bucket); + file.setKey(remoteFilePath); + file.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + List objectsToUploadAsMultipart = new ArrayList(); + objectsToUploadAsMultipart.add(file); + mpUtils.uploadObjects(bucket, s3Service, objectsToUploadAsMultipart, null); + + if (!grantAcl.grantAcl(file)) { + throw new RuntimeException("Failed to set Acl"); } } - - public boolean isBatchUpload() { - return batchUpload; - } - } \ No newline at end of file diff --git a/suro-server/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java b/suro-server/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java index 9a587187..ffcda7a4 100644 --- a/suro-server/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java +++ b/suro-server/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java @@ -246,7 +246,7 @@ public void testAclFailure() throws IOException, ServiceException, InterruptedEx GrantAcl grantAcl = mock(GrantAcl.class); when(grantAcl.grantAcl(any(S3Object.class))).thenReturn(false); sink.open(); - sink.setGrantAcl(grantAcl); + sink.grantAcl = grantAcl; for (Message m : new MessageSetReader(TestConnectionPool.createMessageSet(100000))) { sink.writeTo(new StringMessage(m));