api: Add new ListenBucketNotificationHandler. (#2336)
This API is precursor before implementing `minio lambda` and `mc` continous replication. This new api is an extention to BucketNofication APIs. // Request ``` GET /bucket?notificationARN=arn:minio:lambda:us-east-1:10:minio HTTP/1.1 ... ... ``` // Response ``` {"Records": ...} ... ... ... {"Records": ...} ```master
parent
90c20a8c11
commit
064c51162d
@ -0,0 +1,90 @@ |
||||
package main |
||||
|
||||
import ( |
||||
"bytes" |
||||
"io" |
||||
"io/ioutil" |
||||
"net/http" |
||||
"testing" |
||||
) |
||||
|
||||
// Implement a dummy flush writer.
|
||||
type flushWriter struct { |
||||
io.Writer |
||||
} |
||||
|
||||
// Flush writer is a dummy writer compatible with http.Flusher and http.ResponseWriter.
|
||||
func (f *flushWriter) Flush() {} |
||||
func (f *flushWriter) Write(b []byte) (n int, err error) { return f.Writer.Write(b) } |
||||
func (f *flushWriter) Header() http.Header { return http.Header{} } |
||||
func (f *flushWriter) WriteHeader(code int) {} |
||||
|
||||
func newFlushWriter(writer io.Writer) *flushWriter { |
||||
return &flushWriter{writer} |
||||
} |
||||
|
||||
// Tests write notification code.
|
||||
func TestWriteNotification(t *testing.T) { |
||||
// Initialize a new test config.
|
||||
root, err := newTestConfig("us-east-1") |
||||
if err != nil { |
||||
t.Fatalf("Unable to initialize test config %s", err) |
||||
} |
||||
defer removeAll(root) |
||||
|
||||
var buffer bytes.Buffer |
||||
// Collection of test cases for each event writer.
|
||||
testCases := []struct { |
||||
writer *flushWriter |
||||
event map[string][]NotificationEvent |
||||
err error |
||||
}{ |
||||
// Invalid input argument with writer `nil` - Test - 1
|
||||
{ |
||||
writer: nil, |
||||
event: nil, |
||||
err: errInvalidArgument, |
||||
}, |
||||
// Invalid input argument with event `nil` - Test - 2
|
||||
{ |
||||
writer: newFlushWriter(ioutil.Discard), |
||||
event: nil, |
||||
err: errInvalidArgument, |
||||
}, |
||||
// Unmarshal and write, validate last 5 bytes. - Test - 3
|
||||
{ |
||||
writer: newFlushWriter(&buffer), |
||||
event: map[string][]NotificationEvent{ |
||||
"Records": {newNotificationEvent(eventData{ |
||||
Type: ObjectCreatedPut, |
||||
Bucket: "testbucket", |
||||
ObjInfo: ObjectInfo{ |
||||
Name: "key", |
||||
}, |
||||
ReqParams: map[string]string{ |
||||
"ip": "10.1.10.1", |
||||
}}), |
||||
}, |
||||
}, |
||||
err: nil, |
||||
}, |
||||
} |
||||
// Validates all the testcases for writing notification.
|
||||
for _, testCase := range testCases { |
||||
err := writeNotification(testCase.writer, testCase.event) |
||||
if err != testCase.err { |
||||
t.Errorf("Unable to write notification %s", err) |
||||
} |
||||
// Validates if the ending string has 'crlf'
|
||||
if err == nil && !bytes.HasSuffix(buffer.Bytes(), crlf) { |
||||
buf := buffer.Bytes()[buffer.Len()-5 : 0] |
||||
t.Errorf("Invalid suffix found from the writer last 5 bytes %s, expected `\r\n`", string(buf)) |
||||
} |
||||
// Not printing 'buf' on purpose, validates look for string '10.1.10.1'.
|
||||
if err == nil && !bytes.Contains(buffer.Bytes(), []byte("10.1.10.1")) { |
||||
// Enable when debugging)
|
||||
// fmt.Println(string(buffer.Bytes()))
|
||||
t.Errorf("Requested content couldn't be found, expected `10.1.10.1`") |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,381 @@ |
||||
/* |
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||
* |
||||
* Licensed 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 main |
||||
|
||||
import ( |
||||
"bytes" |
||||
"encoding/xml" |
||||
"errors" |
||||
"fmt" |
||||
"net/url" |
||||
"path" |
||||
"sync" |
||||
"time" |
||||
|
||||
"github.com/Sirupsen/logrus" |
||||
) |
||||
|
||||
// Global event notification queue. This is the queue that would be used to send all notifications.
|
||||
type eventNotifier struct { |
||||
rwMutex *sync.RWMutex |
||||
|
||||
// Collection of 'bucket' and notification config.
|
||||
notificationConfigs map[string]*notificationConfig |
||||
lambdaTargets map[string][]chan []NotificationEvent |
||||
queueTargets map[string]*logrus.Logger |
||||
} |
||||
|
||||
// Represents data to be sent with notification event.
|
||||
type eventData struct { |
||||
Type EventName |
||||
Bucket string |
||||
ObjInfo ObjectInfo |
||||
ReqParams map[string]string |
||||
} |
||||
|
||||
// New notification event constructs a new notification event message from
|
||||
// input request metadata which completed successfully.
|
||||
func newNotificationEvent(event eventData) NotificationEvent { |
||||
/// Construct a new object created event.
|
||||
region := serverConfig.GetRegion() |
||||
tnow := time.Now().UTC() |
||||
sequencer := fmt.Sprintf("%X", tnow.UnixNano()) |
||||
// Following blocks fills in all the necessary details of s3 event message structure.
|
||||
// http://docs.aws.amazon.com/AmazonS3/latest/dev/notification-content-structure.html
|
||||
nEvent := NotificationEvent{ |
||||
EventVersion: "2.0", |
||||
EventSource: "aws:s3", |
||||
AwsRegion: region, |
||||
EventTime: tnow.Format(iso8601Format), |
||||
EventName: event.Type.String(), |
||||
UserIdentity: defaultIdentity(), |
||||
RequestParameters: event.ReqParams, |
||||
ResponseElements: map[string]string{}, |
||||
S3: eventMeta{ |
||||
SchemaVersion: "1.0", |
||||
ConfigurationID: "Config", |
||||
Bucket: bucketMeta{ |
||||
Name: event.Bucket, |
||||
OwnerIdentity: defaultIdentity(), |
||||
ARN: "arn:aws:s3:::" + event.Bucket, |
||||
}, |
||||
}, |
||||
} |
||||
|
||||
escapedObj := url.QueryEscape(event.ObjInfo.Name) |
||||
// For delete object event type, we do not need to set ETag and Size.
|
||||
if event.Type == ObjectRemovedDelete { |
||||
nEvent.S3.Object = objectMeta{ |
||||
Key: escapedObj, |
||||
Sequencer: sequencer, |
||||
} |
||||
return nEvent |
||||
} |
||||
// For all other events we should set ETag and Size.
|
||||
nEvent.S3.Object = objectMeta{ |
||||
Key: escapedObj, |
||||
ETag: event.ObjInfo.MD5Sum, |
||||
Size: event.ObjInfo.Size, |
||||
Sequencer: sequencer, |
||||
} |
||||
// Success.
|
||||
return nEvent |
||||
} |
||||
|
||||
// Fetch the saved queue target.
|
||||
func (en eventNotifier) GetQueueTarget(queueARN string) *logrus.Logger { |
||||
return en.queueTargets[queueARN] |
||||
} |
||||
|
||||
func (en eventNotifier) GetLambdaTarget(lambdaARN string) []chan []NotificationEvent { |
||||
en.rwMutex.RLock() |
||||
defer en.rwMutex.RUnlock() |
||||
return en.lambdaTargets[lambdaARN] |
||||
} |
||||
|
||||
// Set a new lambda target for an input lambda ARN.
|
||||
func (en *eventNotifier) SetLambdaTarget(lambdaARN string, listenerCh chan []NotificationEvent) error { |
||||
en.rwMutex.Lock() |
||||
defer en.rwMutex.Unlock() |
||||
if listenerCh == nil { |
||||
return errors.New("invalid argument") |
||||
} |
||||
en.lambdaTargets[lambdaARN] = append(en.lambdaTargets[lambdaARN], listenerCh) |
||||
return nil |
||||
} |
||||
|
||||
// Remove lambda target for an input lambda ARN.
|
||||
func (en *eventNotifier) RemoveLambdaTarget(lambdaARN string, listenerCh chan []NotificationEvent) { |
||||
en.rwMutex.Lock() |
||||
defer en.rwMutex.Unlock() |
||||
lambdaTarget, ok := en.lambdaTargets[lambdaARN] |
||||
if ok { |
||||
for i, savedListenerCh := range lambdaTarget { |
||||
if listenerCh == savedListenerCh { |
||||
lambdaTarget = append(lambdaTarget[:i], lambdaTarget[i+1:]...) |
||||
if len(lambdaTarget) == 0 { |
||||
delete(en.lambdaTargets, lambdaARN) |
||||
break |
||||
} |
||||
en.lambdaTargets[lambdaARN] = lambdaTarget |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
// Returns true if bucket notification is set for the bucket, false otherwise.
|
||||
func (en eventNotifier) IsBucketNotificationSet(bucket string) bool { |
||||
en.rwMutex.RLock() |
||||
defer en.rwMutex.RUnlock() |
||||
_, ok := en.notificationConfigs[bucket] |
||||
return ok |
||||
} |
||||
|
||||
// Fetch bucket notification config for an input bucket.
|
||||
func (en eventNotifier) GetBucketNotificationConfig(bucket string) *notificationConfig { |
||||
en.rwMutex.RLock() |
||||
defer en.rwMutex.RUnlock() |
||||
return en.notificationConfigs[bucket] |
||||
} |
||||
|
||||
// Set a new notification config for a bucket, this operation will overwrite any previous
|
||||
// notification configs for the bucket.
|
||||
func (en *eventNotifier) SetBucketNotificationConfig(bucket string, notificationCfg *notificationConfig) error { |
||||
en.rwMutex.Lock() |
||||
defer en.rwMutex.Unlock() |
||||
if notificationCfg == nil { |
||||
return errors.New("invalid argument") |
||||
} |
||||
en.notificationConfigs[bucket] = notificationCfg |
||||
return nil |
||||
} |
||||
|
||||
// eventNotify notifies an event to relevant targets based on their
|
||||
// bucket notification configs.
|
||||
func eventNotify(event eventData) { |
||||
// Notifies a new event.
|
||||
// List of events reported through this function are
|
||||
// - s3:ObjectCreated:Put
|
||||
// - s3:ObjectCreated:Post
|
||||
// - s3:ObjectCreated:Copy
|
||||
// - s3:ObjectCreated:CompleteMultipartUpload
|
||||
// - s3:ObjectRemoved:Delete
|
||||
|
||||
nConfig := eventN.GetBucketNotificationConfig(event.Bucket) |
||||
// No bucket notifications enabled, drop the event notification.
|
||||
if len(nConfig.QueueConfigs) == 0 && len(nConfig.LambdaConfigs) == 0 { |
||||
return |
||||
} |
||||
|
||||
// Event type.
|
||||
eventType := event.Type.String() |
||||
|
||||
// Object name.
|
||||
objectName := event.ObjInfo.Name |
||||
|
||||
// Save the notification event to be sent.
|
||||
notificationEvent := []NotificationEvent{newNotificationEvent(event)} |
||||
|
||||
// Validate if the event and object match the queue configs.
|
||||
for _, qConfig := range nConfig.QueueConfigs { |
||||
eventMatch := eventMatch(eventType, qConfig.Events) |
||||
ruleMatch := filterRuleMatch(objectName, qConfig.Filter.Key.FilterRules) |
||||
if eventMatch && ruleMatch { |
||||
targetLog := eventN.GetQueueTarget(qConfig.QueueARN) |
||||
if targetLog != nil { |
||||
targetLog.WithFields(logrus.Fields{ |
||||
"Records": notificationEvent, |
||||
}).Info() |
||||
} |
||||
} |
||||
} |
||||
// Validate if the event and object match the lambda configs.
|
||||
for _, lambdaConfig := range nConfig.LambdaConfigs { |
||||
ruleMatch := filterRuleMatch(objectName, lambdaConfig.Filter.Key.FilterRules) |
||||
eventMatch := eventMatch(eventType, lambdaConfig.Events) |
||||
if eventMatch && ruleMatch { |
||||
targetListeners := eventN.GetLambdaTarget(lambdaConfig.LambdaARN) |
||||
for _, listener := range targetListeners { |
||||
listener <- notificationEvent |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
// loads notifcation config if any for a given bucket, returns back structured notification config.
|
||||
func loadNotificationConfig(bucket string, objAPI ObjectLayer) (*notificationConfig, error) { |
||||
// Construct the notification config path.
|
||||
notificationConfigPath := path.Join(bucketConfigPrefix, bucket, bucketNotificationConfig) |
||||
objInfo, err := objAPI.GetObjectInfo(minioMetaBucket, notificationConfigPath) |
||||
if err != nil { |
||||
// 'notification.xml' not found return 'errNoSuchNotifications'.
|
||||
// This is default when no bucket notifications are found on the bucket.
|
||||
switch err.(type) { |
||||
case ObjectNotFound: |
||||
return nil, errNoSuchNotifications |
||||
} |
||||
// Returns error for other errors.
|
||||
return nil, err |
||||
} |
||||
var buffer bytes.Buffer |
||||
err = objAPI.GetObject(minioMetaBucket, notificationConfigPath, 0, objInfo.Size, &buffer) |
||||
if err != nil { |
||||
// 'notification.xml' not found return 'errNoSuchNotifications'.
|
||||
// This is default when no bucket notifications are found on the bucket.
|
||||
switch err.(type) { |
||||
case ObjectNotFound: |
||||
return nil, errNoSuchNotifications |
||||
} |
||||
// Returns error for other errors.
|
||||
return nil, err |
||||
} |
||||
|
||||
// Unmarshal notification bytes.
|
||||
notificationConfigBytes := buffer.Bytes() |
||||
notificationCfg := ¬ificationConfig{} |
||||
if err = xml.Unmarshal(notificationConfigBytes, ¬ificationCfg); err != nil { |
||||
return nil, err |
||||
} // Successfully marshalled notification configuration.
|
||||
|
||||
// Return success.
|
||||
return notificationCfg, nil |
||||
} |
||||
|
||||
// loads all bucket notifications if present.
|
||||
func loadAllBucketNotifications(objAPI ObjectLayer) (map[string]*notificationConfig, error) { |
||||
// List buckets to proceed loading all notification configuration.
|
||||
buckets, err := objAPI.ListBuckets() |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
configs := make(map[string]*notificationConfig) |
||||
|
||||
// Loads all bucket notifications.
|
||||
for _, bucket := range buckets { |
||||
var nCfg *notificationConfig |
||||
nCfg, err = loadNotificationConfig(bucket.Name, objAPI) |
||||
if err != nil { |
||||
if err == errNoSuchNotifications { |
||||
continue |
||||
} |
||||
return nil, err |
||||
} |
||||
configs[bucket.Name] = nCfg |
||||
} |
||||
|
||||
// Success.
|
||||
return configs, nil |
||||
} |
||||
|
||||
// Loads all queue targets, initializes each queueARNs depending on their config.
|
||||
// Each instance of queueARN registers its own logrus to communicate with the
|
||||
// queue service. QueueARN once initialized is not initialized again for the
|
||||
// same queueARN, instead previous connection is used.
|
||||
func loadAllQueueTargets() (map[string]*logrus.Logger, error) { |
||||
queueTargets := make(map[string]*logrus.Logger) |
||||
// Load all amqp targets, initialize their respective loggers.
|
||||
for accountID, amqpN := range serverConfig.GetAMQP() { |
||||
if !amqpN.Enable { |
||||
continue |
||||
} |
||||
// Construct the queue ARN for AMQP.
|
||||
queueARN := minioSqs + serverConfig.GetRegion() + ":" + accountID + ":" + queueTypeAMQP |
||||
// Queue target if already initialized we move to the next ARN.
|
||||
_, ok := queueTargets[queueARN] |
||||
if ok { |
||||
continue |
||||
} |
||||
// Using accountID we can now initialize a new AMQP logrus instance.
|
||||
amqpLog, err := newAMQPNotify(accountID) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
queueTargets[queueARN] = amqpLog |
||||
} |
||||
// Load redis targets, initialize their respective loggers.
|
||||
for accountID, redisN := range serverConfig.GetRedis() { |
||||
if !redisN.Enable { |
||||
continue |
||||
} |
||||
// Construct the queue ARN for Redis.
|
||||
queueARN := minioSqs + serverConfig.GetRegion() + ":" + accountID + ":" + queueTypeRedis |
||||
// Queue target if already initialized we move to the next ARN.
|
||||
_, ok := queueTargets[queueARN] |
||||
if ok { |
||||
continue |
||||
} |
||||
// Using accountID we can now initialize a new Redis logrus instance.
|
||||
redisLog, err := newRedisNotify(accountID) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
queueTargets[queueARN] = redisLog |
||||
} |
||||
// Load elastic targets, initialize their respective loggers.
|
||||
for accountID, elasticN := range serverConfig.GetElasticSearch() { |
||||
if !elasticN.Enable { |
||||
continue |
||||
} |
||||
// Construct the queue ARN for Elastic.
|
||||
queueARN := minioSqs + serverConfig.GetRegion() + ":" + accountID + ":" + queueTypeElastic |
||||
_, ok := queueTargets[queueARN] |
||||
if ok { |
||||
continue |
||||
} |
||||
// Using accountID we can now initialize a new ElasticSearch logrus instance.
|
||||
elasticLog, err := newElasticNotify(accountID) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
queueTargets[queueARN] = elasticLog |
||||
} |
||||
// Successfully initialized queue targets.
|
||||
return queueTargets, nil |
||||
} |
||||
|
||||
// Global instance of event notification queue.
|
||||
var eventN *eventNotifier |
||||
|
||||
// Initialize event notifier.
|
||||
func initEventNotifier(objAPI ObjectLayer) error { |
||||
if objAPI == nil { |
||||
return errInvalidArgument |
||||
} |
||||
|
||||
// Read all saved bucket notifications.
|
||||
configs, err := loadAllBucketNotifications(objAPI) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
|
||||
// Initializes all queue targets.
|
||||
queueTargets, err := loadAllQueueTargets() |
||||
if err != nil { |
||||
return err |
||||
} |
||||
|
||||
// Inititalize event notifier queue.
|
||||
eventN = &eventNotifier{ |
||||
rwMutex: &sync.RWMutex{}, |
||||
notificationConfigs: configs, |
||||
queueTargets: queueTargets, |
||||
lambdaTargets: make(map[string][]chan []NotificationEvent), |
||||
} |
||||
|
||||
return nil |
||||
} |
@ -0,0 +1,67 @@ |
||||
/* |
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||
* |
||||
* Licensed 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 main |
||||
|
||||
import "testing" |
||||
|
||||
// Tests various forms of inititalization of event notifier.
|
||||
func TestInitEventNotifier(t *testing.T) { |
||||
fs, disk, err := getSingleNodeObjectLayer() |
||||
if err != nil { |
||||
t.Fatal("Unable to initialize FS backend.", err) |
||||
} |
||||
xl, disks, err := getXLObjectLayer() |
||||
if err != nil { |
||||
t.Fatal("Unable to initialize XL backend.", err) |
||||
} |
||||
|
||||
disks = append(disks, disk) |
||||
for _, d := range disks { |
||||
defer removeAll(d) |
||||
} |
||||
|
||||
// Collection of test cases for inititalizing event notifier.
|
||||
testCases := []struct { |
||||
objAPI ObjectLayer |
||||
configs map[string]*notificationConfig |
||||
err error |
||||
}{ |
||||
// Test 1 - invalid arguments.
|
||||
{ |
||||
objAPI: nil, |
||||
err: errInvalidArgument, |
||||
}, |
||||
// Test 2 - valid FS object layer but no bucket notifications.
|
||||
{ |
||||
objAPI: fs, |
||||
err: nil, |
||||
}, |
||||
// Test 3 - valid XL object layer but no bucket notifications.
|
||||
{ |
||||
objAPI: xl, |
||||
err: nil, |
||||
}, |
||||
} |
||||
|
||||
// Validate if event notifier is properly initialized.
|
||||
for i, testCase := range testCases { |
||||
err = initEventNotifier(testCase.objAPI) |
||||
if err != testCase.err { |
||||
t.Errorf("Test %d: Expected %s, but got: %s", i+1, testCase.err, err) |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,136 @@ |
||||
/* |
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||
* |
||||
* Licensed 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 main |
||||
|
||||
import ( |
||||
"errors" |
||||
"strings" |
||||
|
||||
"github.com/minio/minio/pkg/wildcard" |
||||
) |
||||
|
||||
// SQS type.
|
||||
const ( |
||||
// Minio sqs ARN prefix.
|
||||
minioSqs = "arn:minio:sqs:" |
||||
|
||||
// Static string indicating queue type 'amqp'.
|
||||
queueTypeAMQP = "amqp" |
||||
// Static string indicating queue type 'elasticsearch'.
|
||||
queueTypeElastic = "elasticsearch" |
||||
// Static string indicating queue type 'redis'.
|
||||
queueTypeRedis = "redis" |
||||
) |
||||
|
||||
// Lambda type.
|
||||
const ( |
||||
// Minio lambda ARN prefix.
|
||||
minioLambda = "arn:minio:lambda:" |
||||
|
||||
// Static string indicating lambda type 'lambda'.
|
||||
lambdaTypeMinio = "lambda" |
||||
) |
||||
|
||||
var errNotifyNotEnabled = errors.New("requested notifier not enabled") |
||||
|
||||
// Notifier represents collection of supported notification queues.
|
||||
type notifier struct { |
||||
AMQP map[string]amqpNotify `json:"amqp"` |
||||
ElasticSearch map[string]elasticSearchNotify `json:"elasticsearch"` |
||||
Redis map[string]redisNotify `json:"redis"` |
||||
// Add new notification queues.
|
||||
} |
||||
|
||||
// Returns true if queueArn is for an AMQP queue.
|
||||
func isAMQPQueue(sqsArn arnSQS) bool { |
||||
if sqsArn.Type != queueTypeAMQP { |
||||
return false |
||||
} |
||||
amqpL := serverConfig.GetAMQPNotifyByID(sqsArn.AccountID) |
||||
if !amqpL.Enable { |
||||
return false |
||||
} |
||||
// Connect to amqp server to validate.
|
||||
amqpC, err := dialAMQP(amqpL) |
||||
if err != nil { |
||||
errorIf(err, "Unable to connect to amqp service. %#v", amqpL) |
||||
return false |
||||
} |
||||
defer amqpC.Close() |
||||
return true |
||||
} |
||||
|
||||
// Returns true if queueArn is for an Redis queue.
|
||||
func isRedisQueue(sqsArn arnSQS) bool { |
||||
if sqsArn.Type != queueTypeRedis { |
||||
return false |
||||
} |
||||
rNotify := serverConfig.GetRedisNotifyByID(sqsArn.AccountID) |
||||
if !rNotify.Enable { |
||||
return false |
||||
} |
||||
// Connect to redis server to validate.
|
||||
rPool, err := dialRedis(rNotify) |
||||
if err != nil { |
||||
errorIf(err, "Unable to connect to redis service. %#v", rNotify) |
||||
return false |
||||
} |
||||
defer rPool.Close() |
||||
return true |
||||
} |
||||
|
||||
// Returns true if queueArn is for an ElasticSearch queue.
|
||||
func isElasticQueue(sqsArn arnSQS) bool { |
||||
if sqsArn.Type != queueTypeElastic { |
||||
return false |
||||
} |
||||
esNotify := serverConfig.GetElasticSearchNotifyByID(sqsArn.AccountID) |
||||
if !esNotify.Enable { |
||||
return false |
||||
} |
||||
elasticC, err := dialElastic(esNotify) |
||||
if err != nil { |
||||
errorIf(err, "Unable to connect to elasticsearch service %#v", esNotify) |
||||
return false |
||||
} |
||||
defer elasticC.Stop() |
||||
return true |
||||
} |
||||
|
||||
// Match function matches wild cards in 'pattern' for events.
|
||||
func eventMatch(eventType string, events []string) (ok bool) { |
||||
for _, event := range events { |
||||
ok = wildcard.Match(event, eventType) |
||||
if ok { |
||||
break |
||||
} |
||||
} |
||||
return ok |
||||
} |
||||
|
||||
// Filter rule match, matches an object against the filter rules.
|
||||
func filterRuleMatch(object string, frs []filterRule) bool { |
||||
var prefixMatch, suffixMatch = true, true |
||||
for _, fr := range frs { |
||||
if isValidFilterNamePrefix(fr.Name) { |
||||
prefixMatch = strings.HasPrefix(object, fr.Value) |
||||
} else if isValidFilterNameSuffix(fr.Name) { |
||||
suffixMatch = strings.HasSuffix(object, fr.Value) |
||||
} |
||||
} |
||||
return prefixMatch && suffixMatch |
||||
} |
@ -0,0 +1,16 @@ |
||||
// +build linux darwin windows
|
||||
|
||||
package sys |
||||
|
||||
import "testing" |
||||
|
||||
// Test get stats result.
|
||||
func TestGetStats(t *testing.T) { |
||||
stats, err := GetStats() |
||||
if err != nil { |
||||
t.Errorf("Tests: Expected `nil`, Got %s", err) |
||||
} |
||||
if stats.TotalRAM == 0 { |
||||
t.Errorf("Tests: Expected `n > 0`, Got %d", stats.TotalRAM) |
||||
} |
||||
} |
@ -1,204 +0,0 @@ |
||||
/* |
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||
* |
||||
* Licensed 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 main |
||||
|
||||
import ( |
||||
"fmt" |
||||
"net/url" |
||||
"strings" |
||||
"time" |
||||
|
||||
"github.com/Sirupsen/logrus" |
||||
"github.com/minio/minio/pkg/wildcard" |
||||
) |
||||
|
||||
const ( |
||||
minioSqs = "arn:minio:sqs:" |
||||
// Static string indicating queue type 'amqp'.
|
||||
queueTypeAMQP = "1:amqp" |
||||
// Static string indicating queue type 'elasticsearch'.
|
||||
queueTypeElastic = "1:elasticsearch" |
||||
// Static string indicating queue type 'redis'.
|
||||
queueTypeRedis = "1:redis" |
||||
) |
||||
|
||||
// Returns true if queueArn is for an AMQP queue.
|
||||
func isAMQPQueue(sqsArn arnMinioSqs) bool { |
||||
if sqsArn.sqsType != queueTypeAMQP { |
||||
return false |
||||
} |
||||
amqpL := serverConfig.GetAMQPLogger() |
||||
// Connect to amqp server to validate.
|
||||
amqpC, err := dialAMQP(amqpL) |
||||
if err != nil { |
||||
errorIf(err, "Unable to connect to amqp service. %#v", amqpL) |
||||
return false |
||||
} |
||||
defer amqpC.Close() |
||||
return true |
||||
} |
||||
|
||||
// Returns true if queueArn is for an Redis queue.
|
||||
func isRedisQueue(sqsArn arnMinioSqs) bool { |
||||
if sqsArn.sqsType != queueTypeRedis { |
||||
return false |
||||
} |
||||
rLogger := serverConfig.GetRedisLogger() |
||||
// Connect to redis server to validate.
|
||||
rPool, err := dialRedis(rLogger) |
||||
if err != nil { |
||||
errorIf(err, "Unable to connect to redis service. %#v", rLogger) |
||||
return false |
||||
} |
||||
defer rPool.Close() |
||||
return true |
||||
} |
||||
|
||||
// Returns true if queueArn is for an ElasticSearch queue.
|
||||
func isElasticQueue(sqsArn arnMinioSqs) bool { |
||||
if sqsArn.sqsType != queueTypeElastic { |
||||
return false |
||||
} |
||||
esLogger := serverConfig.GetElasticSearchLogger() |
||||
elasticC, err := dialElastic(esLogger) |
||||
if err != nil { |
||||
errorIf(err, "Unable to connect to elasticsearch service %#v", esLogger) |
||||
return false |
||||
} |
||||
defer elasticC.Stop() |
||||
return true |
||||
} |
||||
|
||||
// Match function matches wild cards in 'pattern' for events.
|
||||
func eventMatch(eventType EventName, events []string) (ok bool) { |
||||
for _, event := range events { |
||||
ok = wildcard.Match(event, eventType.String()) |
||||
if ok { |
||||
break |
||||
} |
||||
} |
||||
return ok |
||||
} |
||||
|
||||
// Filter rule match, matches an object against the filter rules.
|
||||
func filterRuleMatch(object string, frs []filterRule) bool { |
||||
var prefixMatch, suffixMatch = true, true |
||||
for _, fr := range frs { |
||||
if isValidFilterNamePrefix(fr.Name) { |
||||
prefixMatch = strings.HasPrefix(object, fr.Value) |
||||
} else if isValidFilterNameSuffix(fr.Name) { |
||||
suffixMatch = strings.HasSuffix(object, fr.Value) |
||||
} |
||||
} |
||||
return prefixMatch && suffixMatch |
||||
} |
||||
|
||||
// NotifyObjectCreatedEvent - notifies a new 's3:ObjectCreated' event.
|
||||
// List of events reported through this function are
|
||||
// - s3:ObjectCreated:Put
|
||||
// - s3:ObjectCreated:Post
|
||||
// - s3:ObjectCreated:Copy
|
||||
// - s3:ObjectCreated:CompleteMultipartUpload
|
||||
func notifyObjectCreatedEvent(nConfig notificationConfig, eventType EventName, bucket string, objInfo ObjectInfo) { |
||||
/// Construct a new object created event.
|
||||
region := serverConfig.GetRegion() |
||||
tnow := time.Now().UTC() |
||||
sequencer := fmt.Sprintf("%X", tnow.UnixNano()) |
||||
// Following blocks fills in all the necessary details of s3 event message structure.
|
||||
// http://docs.aws.amazon.com/AmazonS3/latest/dev/notification-content-structure.html
|
||||
events := []*NotificationEvent{ |
||||
{ |
||||
EventVersion: "2.0", |
||||
EventSource: "aws:s3", |
||||
AwsRegion: region, |
||||
EventTime: tnow.Format(iso8601Format), |
||||
EventName: eventType.String(), |
||||
UserIdentity: defaultIdentity(), |
||||
RequestParameters: map[string]string{}, |
||||
ResponseElements: map[string]string{}, |
||||
S3: eventMeta{ |
||||
SchemaVersion: "1.0", |
||||
ConfigurationID: "Config", |
||||
Bucket: bucketMeta{ |
||||
Name: bucket, |
||||
OwnerIdentity: defaultIdentity(), |
||||
ARN: "arn:aws:s3:::" + bucket, |
||||
}, |
||||
Object: objectMeta{ |
||||
Key: url.QueryEscape(objInfo.Name), |
||||
ETag: objInfo.MD5Sum, |
||||
Size: objInfo.Size, |
||||
Sequencer: sequencer, |
||||
}, |
||||
}, |
||||
}, |
||||
} |
||||
// Notify to all the configured queues.
|
||||
for _, qConfig := range nConfig.QueueConfigurations { |
||||
ruleMatch := filterRuleMatch(objInfo.Name, qConfig.Filter.Key.FilterRules) |
||||
if eventMatch(eventType, qConfig.Events) && ruleMatch { |
||||
log.WithFields(logrus.Fields{ |
||||
"Records": events, |
||||
}).Info() |
||||
} |
||||
} |
||||
} |
||||
|
||||
// NotifyObjectRemovedEvent - notifies a new 's3:ObjectRemoved' event.
|
||||
// List of events reported through this function are
|
||||
// - s3:ObjectRemoved:Delete
|
||||
func notifyObjectDeletedEvent(nConfig notificationConfig, bucket string, object string) { |
||||
region := serverConfig.GetRegion() |
||||
tnow := time.Now().UTC() |
||||
sequencer := fmt.Sprintf("%X", tnow.UnixNano()) |
||||
// Following blocks fills in all the necessary details of s3 event message structure.
|
||||
// http://docs.aws.amazon.com/AmazonS3/latest/dev/notification-content-structure.html
|
||||
events := []*NotificationEvent{ |
||||
{ |
||||
EventVersion: "2.0", |
||||
EventSource: "aws:s3", |
||||
AwsRegion: region, |
||||
EventTime: tnow.Format(iso8601Format), |
||||
EventName: ObjectRemovedDelete.String(), |
||||
UserIdentity: defaultIdentity(), |
||||
RequestParameters: map[string]string{}, |
||||
ResponseElements: map[string]string{}, |
||||
S3: eventMeta{ |
||||
SchemaVersion: "1.0", |
||||
ConfigurationID: "Config", |
||||
Bucket: bucketMeta{ |
||||
Name: bucket, |
||||
OwnerIdentity: defaultIdentity(), |
||||
ARN: "arn:aws:s3:::" + bucket, |
||||
}, |
||||
Object: objectMeta{ |
||||
Key: url.QueryEscape(object), |
||||
Sequencer: sequencer, |
||||
}, |
||||
}, |
||||
}, |
||||
} |
||||
// Notify to all the configured queues.
|
||||
for _, qConfig := range nConfig.QueueConfigurations { |
||||
ruleMatch := filterRuleMatch(object, qConfig.Filter.Key.FilterRules) |
||||
if eventMatch(ObjectRemovedDelete, qConfig.Events) && ruleMatch { |
||||
log.WithFields(logrus.Fields{ |
||||
"Records": events, |
||||
}).Info() |
||||
} |
||||
} |
||||
} |
Loading…
Reference in new issue