api: Do not use sqs for ListenBucketNotification. (#3023)

Add more tests. Fixes #3024
This commit is contained in:
Harshavardhana 2016-10-21 01:25:17 -07:00 committed by GitHub
parent d3aaf50a40
commit ece559afe2
15 changed files with 127 additions and 534 deletions

View file

@ -160,17 +160,6 @@ type NotificationEvent struct {
S3 eventMeta `json:"s3"`
}
// Represents the minio topic type and account id's.
type arnTopic struct {
Type string
AccountID string
}
// Stringer for constructing AWS ARN compatible string.
func (m arnTopic) String() string {
return minioTopic + serverConfig.GetRegion() + ":" + m.AccountID + ":" + m.Type
}
// Represents the minio sqs type and account id's.
type arnSQS struct {
Type string

View file

@ -279,9 +279,11 @@ func (api objectAPIHandlers) ListenBucketNotificationHandler(w http.ResponseWrit
accountID := fmt.Sprintf("%d", time.Now().UTC().UnixNano())
accountARN := fmt.Sprintf(
"arn:minio:sqs:%s:%s:listen-%s",
"%s:%s:%s:%s-%s",
minioTopic,
serverConfig.GetRegion(),
accountID,
snsTypeMinio,
globalMinioAddr,
)
var filterRules []filterRule
@ -338,6 +340,7 @@ func (api objectAPIHandlers) ListenBucketNotificationHandler(w http.ResponseWrit
TopicConfig: *topicCfg,
TargetServer: globalMinioAddr,
}
err = AddBucketListenerConfig(bucket, &lc, objAPI)
if err != nil {
writeErrorResponse(w, r, toAPIErrorCode(err), r.URL.Path)

View file

@ -5,13 +5,13 @@ import (
"bytes"
"encoding/json"
"encoding/xml"
"fmt"
"io"
"io/ioutil"
"net"
"net/http"
"net/http/httptest"
"strconv"
"sync"
"testing"
"github.com/gorilla/mux"
@ -161,41 +161,33 @@ func TestSendBucketNotification(t *testing.T) {
}
}
func testListenBucketNotificationHandler(obj ObjectLayer, instanceType string, t TestErrHandler) {
// Register the API end points with XL/FS object layer.
apiRouter := initTestAPIEndPoints(obj, []string{
func TestListenBucketNotificationHandler(t *testing.T) {
ExecObjectLayerAPITest(t, testListenBucketNotificationHandler, []string{
"ListenBucketNotification",
"PutObject",
})
}
func testListenBucketNotificationHandler(obj ObjectLayer, instanceType, bucketName string, apiRouter http.Handler, credentials credential, t *testing.T) {
mux, ok := apiRouter.(*mux.Router)
if !ok {
t.Fatal("Unable to setup test")
t.Fatal("Invalid mux router found")
}
registerS3PeerRPCRouter(mux)
testServer := httptest.NewServer(apiRouter)
defer testServer.Close()
// initialize the server and obtain the credentials and root.
// credentials are necessary to sign the HTTP request.
rootPath, err := newTestConfig("us-east-1")
if err != nil {
t.Fatalf("Init Test config failed")
}
// remove the root folder after the test ends.
defer removeAll(rootPath)
credentials := serverConfig.GetCredential()
// setup port and minio addr
_, portStr, err := net.SplitHostPort(testServer.Listener.Addr().String())
if err != nil {
t.Fatalf("Initialisation error: %v", err)
t.Fatalf("Initialization error: %v", err)
}
globalMinioPort, err = strconv.Atoi(portStr)
if err != nil {
t.Fatalf("Initialisation error: %v", err)
t.Fatalf("Initialization error: %v", err)
}
globalMinioAddr = fmt.Sprintf(":%d", globalMinioPort)
globalMinioAddr = testServer.Listener.Addr().String()
// initialize the peer client(s)
initGlobalS3Peers([]storageEndPoint{})
@ -212,24 +204,11 @@ func testListenBucketNotificationHandler(obj ObjectLayer, instanceType string, t
}
}
// Initialize global event notifier with mock queue targets.
err = initEventNotifier(obj)
if err != nil {
t.Fatalf("Test %s: Failed to initialize mock event notifier %v",
instanceType, err)
}
var testRec *httptest.ResponseRecorder
var testReq *http.Request
var tErr error
signatureMismatchError := getAPIError(ErrContentSHA256Mismatch)
type testKind int
const (
CheckStatus testKind = iota
InvalidAuth
AsyncHandler
)
tooBigPrefix := string(bytes.Repeat([]byte("a"), 1025))
validEvents := []string{"s3:ObjectCreated:*", "s3:ObjectRemoved:*"}
invalidEvents := []string{"invalidEvent"}
@ -238,15 +217,13 @@ func testListenBucketNotificationHandler(obj ObjectLayer, instanceType string, t
prefixes []string
suffixes []string
events []string
kind testKind
expectedHTTPCode int
expectedAPIError string
}{
// FIXME: Need to find a way to run valid listen bucket notification test case without blocking the unit test.
{randBucket, []string{}, []string{}, invalidEvents, CheckStatus, signatureMismatchError.HTTPStatusCode, ""},
{randBucket, []string{tooBigPrefix}, []string{}, validEvents, CheckStatus, http.StatusBadRequest, ""},
{invalidBucket, []string{}, []string{}, validEvents, CheckStatus, http.StatusBadRequest, ""},
{randBucket, []string{}, []string{}, validEvents, InvalidAuth, signatureMismatchError.HTTPStatusCode, signatureMismatchError.Code},
{randBucket, []string{}, []string{}, invalidEvents, signatureMismatchError.HTTPStatusCode, "InvalidArgument"},
{randBucket, []string{tooBigPrefix}, []string{}, validEvents, http.StatusBadRequest, "InvalidArgument"},
{invalidBucket, []string{}, []string{}, validEvents, http.StatusBadRequest, "InvalidBucketName"},
{randBucket, []string{}, []string{}, validEvents, signatureMismatchError.HTTPStatusCode, signatureMismatchError.Code},
}
for i, test := range testCases {
@ -258,51 +235,42 @@ func testListenBucketNotificationHandler(obj ObjectLayer, instanceType string, t
t.Fatalf("%s: Failed to create HTTP testRequest for ListenBucketNotification: <ERROR> %v", instanceType, tErr)
}
// Set X-Amz-Content-SHA256 in header different from what was used to calculate Signature.
if test.kind == InvalidAuth {
// Triggering a authentication type check failure.
if test.expectedAPIError == "XAmzContentSHA256Mismatch" {
// Triggering a authentication failure.
testReq.Header.Set("x-amz-content-sha256", "somethingElse")
}
if test.kind == AsyncHandler {
go apiRouter.ServeHTTP(testRec, testReq)
} else {
apiRouter.ServeHTTP(testRec, testReq)
switch test.kind {
case InvalidAuth:
rspBytes, rErr := ioutil.ReadAll(testRec.Body)
if rErr != nil {
t.Errorf("Test %d: %s: Failed to read response body: <ERROR> %v", i+1, instanceType, rErr)
}
var errCode APIError
xErr := xml.Unmarshal(rspBytes, &errCode)
if xErr != nil {
t.Errorf("Test %d: %s: Failed to unmarshal error XML: <ERROR> %v", i+1, instanceType, xErr)
apiRouter.ServeHTTP(testRec, testReq)
rspBytes, rErr := ioutil.ReadAll(testRec.Body)
if rErr != nil {
t.Errorf("Test %d: %s: Failed to read response body: <ERROR> %v", i+1, instanceType, rErr)
}
var errXML APIErrorResponse
xErr := xml.Unmarshal(rspBytes, &errXML)
if xErr != nil {
t.Errorf("Test %d: %s: Failed to unmarshal error XML: <ERROR> %v", i+1, instanceType, xErr)
}
if errXML.Code != test.expectedAPIError {
t.Errorf("Test %d: %s: Expected error code %s but received %s: <ERROR> %v", i+1,
instanceType, test.expectedAPIError, errXML.Code, err)
}
if errCode.Code != test.expectedAPIError {
t.Errorf("Test %d: %s: Expected error code %s but received %s: <ERROR> %v", i+1,
instanceType, test.expectedAPIError, errCode.Code, err)
}
fallthrough
case CheckStatus:
if testRec.Code != test.expectedHTTPCode {
t.Errorf("Test %d: %s: expected HTTP code %d, but received %d: <ERROR> %v",
i+1, instanceType, test.expectedHTTPCode, testRec.Code, err)
}
}
}
if testRec.Code != test.expectedHTTPCode {
t.Errorf("Test %d: %s: expected HTTP code %d, but received %d: <ERROR> %v",
i+1, instanceType, test.expectedHTTPCode, testRec.Code, err)
}
}
// Nil Object layer
nilAPIRouter := initTestAPIEndPoints(nil, []string{
"PutBucketNotification",
"ListenBucketNotification",
})
testRec = httptest.NewRecorder()
testReq, tErr = newTestSignedRequestV4("GET",
getListenBucketNotificationURL("", randBucket, []string{}, []string{"*.jpg"}, []string{"s3:ObjectCreated:*", "s3:ObjectRemoved:*"}),
0, nil, credentials.AccessKeyID, credentials.SecretAccessKey)
getListenBucketNotificationURL("", randBucket, []string{},
[]string{"*.jpg"}, []string{
"s3:ObjectCreated:*",
"s3:ObjectRemoved:*",
}), 0, nil, credentials.AccessKeyID, credentials.SecretAccessKey)
if tErr != nil {
t.Fatalf("%s: Failed to create HTTP testRequest for ListenBucketNotification: <ERROR> %v", instanceType, tErr)
}
@ -311,10 +279,56 @@ func testListenBucketNotificationHandler(obj ObjectLayer, instanceType string, t
t.Errorf("Test %d: %s: expected HTTP code %d, but received %d: <ERROR> %v",
1, instanceType, http.StatusServiceUnavailable, testRec.Code, err)
}
}
func TestListenBucketNotificationHandler(t *testing.T) {
ExecObjectLayerTest(t, testListenBucketNotificationHandler)
testRec = httptest.NewRecorder()
testReq, tErr = newTestSignedRequestV4("GET",
getListenBucketNotificationURL("", randBucket, []string{}, []string{}, validEvents),
0, nil, credentials.AccessKeyID, credentials.SecretAccessKey)
if tErr != nil {
t.Fatalf("%s: Failed to create HTTP testRequest for ListenBucketNotification: <ERROR> %v", instanceType, tErr)
}
globalObjLayerMutex.Lock()
globalObjectAPI = obj
globalObjLayerMutex.Unlock()
go apiRouter.ServeHTTP(testRec, testReq)
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
rec := httptest.NewRecorder()
buf := bytes.NewReader([]byte("hello, world"))
req, rerr := newTestSignedRequestV4("PUT", getPutObjectURL("", randBucket, "jeezus"),
int64(buf.Len()), buf, credentials.AccessKeyID, credentials.SecretAccessKey)
if rerr != nil {
t.Fatalf("%s: Failed to create HTTP testRequest for ListenBucketNotification: <ERROR> %v", instanceType, rerr)
}
apiRouter.ServeHTTP(rec, req)
if rec.Code != http.StatusOK {
t.Fatalf("Unexpected http reply %d should be %d", rec.Code, http.StatusOK)
}
}()
wg.Wait()
bio := bufio.NewScanner(testRec.Body)
// Unmarshal each line, returns marshalled values.
for bio.Scan() {
var notificationInfo struct {
Records []NotificationEvent
}
if err = json.Unmarshal(bio.Bytes(), &notificationInfo); err != nil {
t.Fatalf("%s: Unable to marshal: <ERROR> %v", instanceType, err)
}
// Send notifications on channel only if there are events received.
if len(notificationInfo.Records) == 0 {
t.Fatalf("%s: Expected notification events, received none", instanceType)
}
}
// Look for any underlying errors.
if err = bio.Err(); err != nil {
t.Fatalf("%s: Server connection closed prematurely %s", instanceType, err)
}
}
func testRemoveNotificationConfig(obj ObjectLayer, instanceType string, t TestErrHandler) {

View file

@ -126,21 +126,6 @@ func checkQueueARN(queueARN string) APIErrorCode {
return checkARN(queueARN, minioSqs)
}
// checkTopicARN - check if the topic arn is valid.
func checkTopicARN(topicARN string) APIErrorCode {
return checkARN(topicARN, minioTopic)
}
// Returns true if the topicARN is for an Minio sns listen type.
func isMinioSNS(topicARN arnTopic) bool {
return strings.HasSuffix(topicARN.Type, snsTypeMinio)
}
// Validate if we recognize the topic type.
func isValidTopic(topicARN arnTopic) bool {
return isMinioSNS(topicARN) // Is minio topic?.
}
// Validates account id for input queue ARN.
func isValidQueueID(queueARN string) bool {
// Unmarshals QueueARN into structured object.
@ -192,34 +177,6 @@ func checkQueueConfig(qConfig queueConfig) APIErrorCode {
return ErrNone
}
// Check - validates queue configuration and returns error if any.
func checkTopicConfig(tConfig topicConfig) APIErrorCode {
// Check queue arn is valid.
if s3Error := checkTopicARN(tConfig.TopicARN); s3Error != ErrNone {
return s3Error
}
// Unmarshals QueueARN into structured object.
topicARN := unmarshalTopicARN(tConfig.TopicARN)
// Validate if topicARN requested any of the known supported queues.
if !isValidTopic(topicARN) {
return ErrARNNotification
}
// Check if valid events are set in queue config.
if s3Error := checkEvents(tConfig.Events); s3Error != ErrNone {
return s3Error
}
// Check if valid filters are set in queue config.
if s3Error := checkFilterRules(tConfig.Filter.Key.FilterRules); s3Error != ErrNone {
return s3Error
}
// Success.
return ErrNone
}
// Validates all incoming queue configs, checkQueueConfig validates if the
// input fields for each queues is not malformed and has valid configuration
// information. If validation fails bucket notifications are not enabled.
@ -233,19 +190,6 @@ func validateQueueConfigs(queueConfigs []queueConfig) APIErrorCode {
return ErrNone
}
// Validates all incoming topic configs, checkTopicConfig validates if the
// input fields for each queues is not malformed and has valid configuration
// information. If validation fails bucket notifications are not enabled.
func validateTopicConfigs(topicConfigs []topicConfig) APIErrorCode {
for _, tConfig := range topicConfigs {
if s3Error := checkTopicConfig(tConfig); s3Error != ErrNone {
return s3Error
}
}
// Success.
return ErrNone
}
// Check all the queue configs for any duplicates.
func checkDuplicateQueueConfigs(configs []queueConfig) APIErrorCode {
var queueConfigARNS []string
@ -285,23 +229,6 @@ func validateNotificationConfig(nConfig notificationConfig) APIErrorCode {
return ErrNone
}
// Unmarshals input value of AWS ARN format into minioTopic object.
// Returned value represents minio topic type, currently supported are
// - listen
func unmarshalTopicARN(topicARN string) arnTopic {
topic := arnTopic{}
if !strings.HasPrefix(topicARN, minioTopic+serverConfig.GetRegion()+":") {
return topic
}
topicType := strings.TrimPrefix(topicARN, minioTopic+serverConfig.GetRegion()+":")
switch {
case strings.HasSuffix(topicType, snsTypeMinio):
topic.Type = snsTypeMinio
} // Add more topic here.
topic.AccountID = strings.TrimSuffix(topicType, ":"+topic.Type)
return topic
}
// Unmarshals input value of AWS ARN format into minioSqs object.
// Returned value represents minio sqs types, currently supported are
// - amqp

View file

@ -216,79 +216,6 @@ func TestValidEvents(t *testing.T) {
}
}
// Tests topic arn validation.
func TestTopicARN(t *testing.T) {
rootPath, err := newTestConfig("us-east-1")
if err != nil {
t.Fatalf("unable initialize config file, %s", err)
}
defer removeAll(rootPath)
testCases := []struct {
topicARN string
errCode APIErrorCode
}{
// Valid minio topic with '1' account id.
{
topicARN: "arn:minio:sns:us-east-1:1:minio",
errCode: ErrNone,
},
// Valid minio topic with '10' account id.
{
topicARN: "arn:minio:sns:us-east-1:10:minio",
errCode: ErrNone,
},
// Invalid empty topic arn.
{
topicARN: "",
errCode: ErrARNNotification,
},
// Invalid notification service type.
{
topicARN: "arn:minio:sqs:us-east-1:1:listen",
errCode: ErrARNNotification,
},
// Invalid region 'us-west-1' in queue arn.
{
topicARN: "arn:minio:sns:us-west-1:1:listen",
errCode: ErrRegionNotification,
},
// Empty topic account id is invalid.
{
topicARN: "arn:minio:sns:us-east-1::listen",
errCode: ErrARNNotification,
},
// Empty topic account name is invalid.
{
topicARN: "arn:minio:sns:us-east-1:10:",
errCode: ErrARNNotification,
},
// Empty topic account id and account name is invalid.
{
topicARN: "arn:minio:sns:us-east-1::",
errCode: ErrARNNotification,
},
// Missing topic id and separator missing at the end in topic arn.
{
topicARN: "arn:minio:sns:us-east-1:listen",
errCode: ErrARNNotification,
},
// Missing topic id and empty string at the end in topic arn.
{
topicARN: "arn:minio:sns:us-east-1:",
errCode: ErrARNNotification,
},
}
// Validate all topics.
for i, testCase := range testCases {
errCode := checkTopicARN(testCase.topicARN)
if testCase.errCode != errCode {
t.Errorf("Test %d: Expected \"%d\", got \"%d\"", i+1, testCase.errCode, errCode)
}
}
}
// Tests queue arn validation.
func TestQueueARN(t *testing.T) {
rootPath, err := newTestConfig("us-east-1")
@ -367,55 +294,8 @@ func TestQueueARN(t *testing.T) {
}
}
// Test unmarshal topic arn.
func TestUnmarshalTopicARN(t *testing.T) {
rootPath, err := newTestConfig("us-east-1")
if err != nil {
t.Fatalf("unable initialize config file, %s", err)
}
defer removeAll(rootPath)
testCases := []struct {
topicARN string
Type string
}{
// Valid minio topic arn.
{
topicARN: "arn:minio:sns:us-east-1:1:listen",
Type: "listen",
},
// Invalid empty topic arn.
{
topicARN: "",
Type: "",
},
// Invalid region 'us-west-1' in topic arn.
{
topicARN: "arn:minio:sns:us-west-1:1:listen",
Type: "",
},
// Partial topic arn.
{
topicARN: "arn:minio:sns:",
Type: "",
},
// Invalid topic service value.
{
topicARN: "arn:minio:sns:us-east-1:1:*",
Type: "",
},
}
for i, testCase := range testCases {
topic := unmarshalTopicARN(testCase.topicARN)
if testCase.Type != topic.Type {
t.Errorf("Test %d: Expected \"%s\", got \"%s\"", i+1, testCase.Type, topic.Type)
}
}
}
// Test unmarshal queue arn.
func TestUnmarshalSqsARN(t *testing.T) {
func TestUnmarshalSQSARN(t *testing.T) {
rootPath, err := newTestConfig("us-east-1")
if err != nil {
t.Fatalf("unable initialize config file, %s", err)

View file

@ -18,6 +18,7 @@ package cmd
import (
"bytes"
"encoding/hex"
"encoding/json"
"encoding/xml"
"fmt"
@ -411,8 +412,8 @@ func persistNotificationConfig(bucket string, ncfg *notificationConfig, obj Obje
// build path
ncPath := path.Join(bucketConfigPrefix, bucket, bucketNotificationConfig)
// write object to path
_, err = obj.PutObject(minioMetaBucket, ncPath, int64(len(buf)),
bytes.NewReader(buf), nil, "")
sha256Sum := hex.EncodeToString(sum256(buf))
_, err = obj.PutObject(minioMetaBucket, ncPath, int64(len(buf)), bytes.NewReader(buf), nil, sha256Sum)
if err != nil {
errorIf(err, "Unable to write bucket notification configuration.")
return err
@ -437,16 +438,15 @@ func persistListenerConfig(bucket string, lcfg []listenerConfig, obj ObjectLayer
// build path
lcPath := path.Join(bucketConfigPrefix, bucket, bucketListenerConfig)
// write object to path
_, err = obj.PutObject(minioMetaBucket, lcPath, int64(len(buf)),
bytes.NewReader(buf), nil, "")
sha256Sum := hex.EncodeToString(sum256(buf))
_, err = obj.PutObject(minioMetaBucket, lcPath, int64(len(buf)), bytes.NewReader(buf), nil, sha256Sum)
if err != nil {
errorIf(err, "Unable to write bucket listener configuration to object layer.")
}
return err
}
// Remove listener configuration from storage layer. Used when a
// bucket is deleted.
// Remove listener configuration from storage layer. Used when a bucket is deleted.
func removeListenerConfig(bucket string, obj ObjectLayer) error {
// make the path
lcPath := path.Join(bucketConfigPrefix, bucket, bucketListenerConfig)

View file

@ -331,7 +331,6 @@ func TestInitEventNotifier(t *testing.T) {
func TestListenBucketNotification(t *testing.T) {
s := TestPeerRPCServerData{serverType: "XL"}
// setup and teardown
s.Setup(t)
defer s.TearDown()
@ -347,9 +346,14 @@ func TestListenBucketNotification(t *testing.T) {
t.Fatal("Unexpected error:", err)
}
listenARN := "arn:minio:sns:us-east-1:1:listen-" + globalMinioAddr
listenARN := fmt.Sprintf("%s:%s:1:%s-%s",
minioTopic,
serverConfig.GetRegion(),
snsTypeMinio,
s.testServer.Server.Listener.Addr(),
)
lcfg := listenerConfig{
topicConfig{
TopicConfig: topicConfig{
ServiceConfig{
[]string{"s3:ObjectRemoved:*", "s3:ObjectCreated:*"},
filterStruct{},
@ -357,7 +361,7 @@ func TestListenBucketNotification(t *testing.T) {
},
listenARN,
},
globalMinioAddr,
TargetServer: globalMinioAddr,
}
// write listener config to storage layer

View file

@ -26,7 +26,6 @@ type statusType string
const (
runningStatus statusType = "Running"
readyStatus statusType = "Ready"
blockedStatus statusType = "Blocked"
)

View file

@ -347,25 +347,6 @@ func validateDisks(endPoints []storageEndPoint, ignoredEndPoints []storageEndPoi
return storageDisks
}
// Extract port number from address address should be of the form host:port.
func getPort(address string) int {
_, portStr, _ := net.SplitHostPort(address)
// If port empty, default to port '80'
if portStr == "" {
portStr = "80"
// if SSL is enabled, choose port as "443" instead.
if isSSL() {
portStr = "443"
}
}
// Return converted port number.
portInt, err := strconv.Atoi(portStr)
fatalIf(err, "Invalid port number.")
return portInt
}
// Returns if slice of disks is a distributed setup.
func isDistributedSetup(eps []storageEndPoint) (isDist bool) {
// Port to connect to for the lock servers in a distributed setup.

View file

@ -20,7 +20,6 @@ import (
"flag"
"net/http"
"os"
"path/filepath"
"runtime"
"testing"
@ -186,55 +185,6 @@ func TestCheckServerSyntax(t *testing.T) {
}
}
func TestGetPort(t *testing.T) {
root, err := newTestConfig("us-east-1")
if err != nil {
t.Fatal("failed to create test config")
}
defer removeAll(root)
testCases := []struct {
addr string
ssl bool
expectedPort int
}{
{"localhost:1234", true, 1234},
{"localhost:1234", false, 1234},
{"localhost", true, 443},
{"localhost", false, 80},
}
certFile := filepath.Join(mustGetCertsPath(), globalMinioCertFile)
keyFile := filepath.Join(mustGetCertsPath(), globalMinioKeyFile)
err = os.MkdirAll(filepath.Dir(certFile), 0755)
if err != nil {
t.Fatalf("Couldn't create certs directory.")
}
for i, test := range testCases {
if test.ssl {
cFile, cErr := os.Create(certFile)
if cErr != nil {
t.Fatalf("Failed to create cert file %s", certFile)
}
cFile.Close()
tFile, tErr := os.Create(keyFile)
if tErr != nil {
t.Fatalf("Failed to create key file %s", keyFile)
}
tFile.Close()
}
port := getPort(test.addr)
if port != test.expectedPort {
t.Errorf("Test %d expected port %d but received %d", i+1, test.expectedPort, port)
}
if test.ssl {
os.Remove(certFile)
os.Remove(keyFile)
}
}
}
func TestIsDistributedSetup(t *testing.T) {
var testCases []struct {
disks []string

View file

@ -22,12 +22,6 @@ import (
"syscall"
)
// Represents a type of an exit func which will be invoked upon service signal.
type onExitFunc func(err error)
// Represents a type for all the the callback functions invoked upon service signal.
type cleanupOnExitFunc func() error
// Type of service signals currently supported.
type serviceSignal int

View file

@ -1575,7 +1575,7 @@ func prepareXLStorageDisks(t *testing.T) ([]StorageAPI, []string) {
// initializes the specified API endpoints for the tests.
// initialies the root and returns its path.
// return credentials.
func initAPIHandlerTest(obj ObjectLayer, endPoints []string) (bucketName, rootPath string, apiRouter http.Handler, err error) {
func initAPIHandlerTest(obj ObjectLayer, endPoints []string) (bucketName string, apiRouter http.Handler, err error) {
// get random bucket name.
bucketName = getRandomBucketName()
@ -1583,19 +1583,12 @@ func initAPIHandlerTest(obj ObjectLayer, endPoints []string) (bucketName, rootPa
err = obj.MakeBucket(bucketName)
if err != nil {
// failed to create newbucket, return err.
return "", "", nil, err
return "", nil, err
}
// Register the API end points with XL/FS object layer.
// Registering only the GetObject handler.
apiRouter = initTestAPIEndPoints(obj, endPoints)
// initialize the server and obtain the credentials and root.
// credentials are necessary to sign the HTTP request.
rootPath, err = newTestConfig("us-east-1")
if err != nil {
return "", "", nil, err
}
return bucketName, rootPath, apiRouter, nil
return bucketName, apiRouter, nil
}
// ExecObjectLayerAPIAnonTest - Helper function to validate object Layer API handler
@ -1768,11 +1761,17 @@ func ExecObjectLayerAPINilTest(t TestErrHandler, bucketName, objectName, instanc
// ExecObjectLayerAPITest - executes object layer API tests.
// Creates single node and XL ObjectLayer instance, registers the specified API end points and runs test for both the layers.
func ExecObjectLayerAPITest(t *testing.T, objAPITest objAPITestType, endPoints []string) {
// initialize the server and obtain the credentials and root.
// credentials are necessary to sign the HTTP request.
rootPath, err := newTestConfig("us-east-1")
if err != nil {
t.Fatalf("Unable to initialize server config. %s", err)
}
objLayer, fsDir, err := prepareFS()
if err != nil {
t.Fatalf("Initialization of object layer failed for single node setup: %s", err)
}
bucketFS, fsRoot, fsAPIRouter, err := initAPIHandlerTest(objLayer, endPoints)
bucketFS, fsAPIRouter, err := initAPIHandlerTest(objLayer, endPoints)
if err != nil {
t.Fatalf("Initialzation of API handler tests failed: <ERROR> %s", err)
}
@ -1784,7 +1783,7 @@ func ExecObjectLayerAPITest(t *testing.T, objAPITest objAPITestType, endPoints [
if err != nil {
t.Fatalf("Initialization of object layer failed for XL setup: %s", err)
}
bucketXL, xlRoot, xlAPIRouter, err := initAPIHandlerTest(objLayer, endPoints)
bucketXL, xlAPIRouter, err := initAPIHandlerTest(objLayer, endPoints)
if err != nil {
t.Fatalf("Initialzation of API handler tests failed: <ERROR> %s", err)
}
@ -1792,7 +1791,7 @@ func ExecObjectLayerAPITest(t *testing.T, objAPITest objAPITestType, endPoints [
// Executing the object layer tests for XL.
objAPITest(objLayer, xLTestStr, bucketXL, xlAPIRouter, credentials, t)
// clean up the temporary test backend.
removeRoots(append(xlDisks, fsDir, fsRoot, xlRoot))
removeRoots(append(xlDisks, fsDir, rootPath))
}
// function to be passed to ExecObjectLayerAPITest, for executing object layr API handler tests.
@ -1808,6 +1807,14 @@ type objTestDiskNotFoundType func(obj ObjectLayer, instanceType string, dirs []s
// ExecObjectLayerTest - executes object layer tests.
// Creates single node and XL ObjectLayer instance and runs test for both the layers.
func ExecObjectLayerTest(t TestErrHandler, objTest objTestType) {
// initialize the server and obtain the credentials and root.
// credentials are necessary to sign the HTTP request.
rootPath, err := newTestConfig("us-east-1")
if err != nil {
t.Fatal("Unexpected error", err)
}
defer removeAll(rootPath)
objLayer, fsDir, err := prepareFS()
if err != nil {
t.Fatalf("Initialization of object layer failed for single node setup: %s", err)

View file

@ -21,11 +21,8 @@ import (
"encoding/xml"
"fmt"
"io"
"net"
"net/http"
"os"
"path/filepath"
"runtime"
"strings"
"encoding/json"
@ -81,27 +78,6 @@ func checkDuplicateEndPoints(list []storageEndPoint) error {
return checkDuplicateStrings(strs)
}
// splits network path into its components Address and Path.
func splitNetPath(networkPath string) (netAddr, netPath string, err error) {
if runtime.GOOS == "windows" {
if volumeName := filepath.VolumeName(networkPath); volumeName != "" {
return "", networkPath, nil
}
}
networkParts := strings.SplitN(networkPath, ":", 2)
switch {
case len(networkParts) == 1:
return "", networkPath, nil
case networkParts[1] == "":
return "", "", &net.AddrError{Err: "Missing path in network path", Addr: networkPath}
case networkParts[0] == "":
return "", "", &net.AddrError{Err: "Missing address in network path", Addr: networkPath}
case !filepath.IsAbs(networkParts[1]):
return "", "", &net.AddrError{Err: "Network path should be absolute", Addr: networkPath}
}
return networkParts[0], networkParts[1], nil
}
// Find local node through the command line arguments. Returns in
// `host:port` format.
func getLocalAddress(srvCmdConfig serverCmdConfig) string {

View file

@ -1,61 +0,0 @@
// +build !windows
/*
* 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 cmd
import (
"net"
"testing"
)
// Test for splitNetPath
func TestSplitNetPath(t *testing.T) {
testCases := []struct {
networkPath string
netAddr string
netPath string
err error
}{
// Invalid cases 1-5.
{"10.1.10.1:", "", "", &net.AddrError{Err: "Missing path in network path", Addr: "10.1.10.1:"}},
{"10.1.10.1:../1", "", "", &net.AddrError{Err: "Network path should be absolute", Addr: "10.1.10.1:../1"}},
{":/tmp/1", "", "", &net.AddrError{Err: "Missing address in network path", Addr: ":/tmp/1"}},
{"10.1.10.1:disk/1", "", "", &net.AddrError{Err: "Network path should be absolute", Addr: "10.1.10.1:disk/1"}},
{"10.1.10.1:\\path\\test", "", "", &net.AddrError{Err: "Network path should be absolute", Addr: "10.1.10.1:\\path\\test"}},
// Valid cases 6-8
{"10.1.10.1", "", "10.1.10.1", nil},
{"10.1.10.1://", "10.1.10.1", "//", nil},
{"10.1.10.1:/disk/1", "10.1.10.1", "/disk/1", nil},
}
for i, test := range testCases {
receivedAddr, receivedPath, receivedErr := splitNetPath(test.networkPath)
if receivedAddr != test.netAddr {
t.Errorf("Test case %d: Expected: %s, Received: %s", i+1, test.netAddr, receivedAddr)
}
if receivedPath != test.netPath {
t.Errorf("Test case %d: Expected: %s, Received: %s", i+1, test.netPath, receivedPath)
}
if test.err != nil {
if receivedErr == nil || receivedErr.Error() != test.err.Error() {
t.Errorf("Test case %d: Expected: %v, Received: %v", i+1, test.err, receivedErr)
}
}
}
}

View file

@ -1,70 +0,0 @@
// +build windows
/*
* Minio Cloud Storage, (C) 2015 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 cmd
import (
"net"
"testing"
)
// Test for splitNetPath
func TestSplitNetPath(t *testing.T) {
testCases := []struct {
networkPath string
netAddr string
netPath string
err error
}{
// Invalid cases 1-8.
{":C:", "", "", &net.AddrError{Err: "Missing address in network path", Addr: ":C:"}},
{"10.1.10.1:", "", "", &net.AddrError{Err: "Missing path in network path", Addr: "10.1.10.1:"}},
{"10.1.10.1:C", "", "", &net.AddrError{Err: "Network path should be absolute", Addr: "10.1.10.1:C"}},
{"10.1.10.1:C:", "", "", &net.AddrError{Err: "Network path should be absolute", Addr: "10.1.10.1:C:"}},
{"10.1.10.1:C:../path", "", "", &net.AddrError{Err: "Network path should be absolute", Addr: "10.1.10.1:C:../path"}},
{"10.1.10.1:C:tmp/1", "", "", &net.AddrError{Err: "Network path should be absolute", Addr: "10.1.10.1:C:tmp/1"}},
{"10.1.10.1::C:\\path\\test", "", "", &net.AddrError{
Err: "Network path should be absolute",
Addr: "10.1.10.1::C:\\path\\test",
}},
{"10.1.10.1:\\path\\test", "", "", &net.AddrError{
Err: "Network path should be absolute",
Addr: "10.1.10.1:\\path\\test",
}},
// Valid cases 9-11.
{"10.1.10.1:C:\\path\\test", "10.1.10.1", "C:\\path\\test", nil},
{"C:\\path\\test", "", "C:\\path\\test", nil},
{`10.1.10.1:\\?\UNC\path\test`, "10.1.10.1", `\\?\UNC\path\test`, nil},
}
for i, test := range testCases {
receivedAddr, receivedPath, receivedErr := splitNetPath(test.networkPath)
if receivedAddr != test.netAddr {
t.Errorf("Test case %d: Expected: %s, Received: %s", i+1, test.netAddr, receivedAddr)
}
if receivedPath != test.netPath {
t.Errorf("Test case %d: Expected: %s, Received: %s", i+1, test.netPath, receivedPath)
}
if test.err != nil {
if receivedErr == nil || receivedErr.Error() != test.err.Error() {
t.Errorf("Test case %d: Expected: %v, Received: %v", i+1, test.err, receivedErr)
}
}
}
}