cassandra_store.go 6.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223
  1. package cassandra
  2. import (
  3. "context"
  4. "fmt"
  5. "github.com/gocql/gocql"
  6. "time"
  7. "github.com/seaweedfs/seaweedfs/weed/filer"
  8. "github.com/seaweedfs/seaweedfs/weed/glog"
  9. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  10. "github.com/seaweedfs/seaweedfs/weed/util"
  11. )
  12. func init() {
  13. filer.Stores = append(filer.Stores, &CassandraStore{})
  14. }
  15. type CassandraStore struct {
  16. cluster *gocql.ClusterConfig
  17. session *gocql.Session
  18. superLargeDirectoryHash map[string]string
  19. }
  20. func (store *CassandraStore) GetName() string {
  21. return "cassandra"
  22. }
  23. func (store *CassandraStore) Initialize(configuration util.Configuration, prefix string) (err error) {
  24. return store.initialize(
  25. configuration.GetString(prefix+"keyspace"),
  26. configuration.GetStringSlice(prefix+"hosts"),
  27. configuration.GetString(prefix+"username"),
  28. configuration.GetString(prefix+"password"),
  29. configuration.GetStringSlice(prefix+"superLargeDirectories"),
  30. configuration.GetString(prefix+"localDC"),
  31. configuration.GetInt(prefix+"connection_timeout_millisecond"),
  32. )
  33. }
  34. func (store *CassandraStore) isSuperLargeDirectory(dir string) (dirHash string, isSuperLargeDirectory bool) {
  35. dirHash, isSuperLargeDirectory = store.superLargeDirectoryHash[dir]
  36. return
  37. }
  38. func (store *CassandraStore) initialize(keyspace string, hosts []string, username string, password string, superLargeDirectories []string, localDC string, timeout int) (err error) {
  39. store.cluster = gocql.NewCluster(hosts...)
  40. if username != "" && password != "" {
  41. store.cluster.Authenticator = gocql.PasswordAuthenticator{Username: username, Password: password}
  42. }
  43. store.cluster.Keyspace = keyspace
  44. store.cluster.Timeout = time.Duration(timeout) * time.Millisecond
  45. glog.V(0).Infof("timeout = %d", timeout)
  46. fallback := gocql.RoundRobinHostPolicy()
  47. if localDC != "" {
  48. fallback = gocql.DCAwareRoundRobinPolicy(localDC)
  49. }
  50. store.cluster.PoolConfig.HostSelectionPolicy = gocql.TokenAwareHostPolicy(fallback)
  51. store.cluster.Consistency = gocql.LocalQuorum
  52. store.session, err = store.cluster.CreateSession()
  53. if err != nil {
  54. glog.V(0).Infof("Failed to open cassandra store, hosts %v, keyspace %s", hosts, keyspace)
  55. }
  56. // set directory hash
  57. store.superLargeDirectoryHash = make(map[string]string)
  58. existingHash := make(map[string]string)
  59. for _, dir := range superLargeDirectories {
  60. // adding dir hash to avoid duplicated names
  61. dirHash := util.Md5String([]byte(dir))[:4]
  62. store.superLargeDirectoryHash[dir] = dirHash
  63. if existingDir, found := existingHash[dirHash]; found {
  64. glog.Fatalf("directory %s has the same hash as %s", dir, existingDir)
  65. }
  66. existingHash[dirHash] = dir
  67. }
  68. return
  69. }
  70. func (store *CassandraStore) BeginTransaction(ctx context.Context) (context.Context, error) {
  71. return ctx, nil
  72. }
  73. func (store *CassandraStore) CommitTransaction(ctx context.Context) error {
  74. return nil
  75. }
  76. func (store *CassandraStore) RollbackTransaction(ctx context.Context) error {
  77. return nil
  78. }
  79. func (store *CassandraStore) InsertEntry(ctx context.Context, entry *filer.Entry) (err error) {
  80. dir, name := entry.FullPath.DirAndName()
  81. if dirHash, ok := store.isSuperLargeDirectory(dir); ok {
  82. dir, name = dirHash+name, ""
  83. }
  84. meta, err := entry.EncodeAttributesAndChunks()
  85. if err != nil {
  86. return fmt.Errorf("encode %s: %s", entry.FullPath, err)
  87. }
  88. if len(entry.GetChunks()) > filer.CountEntryChunksForGzip {
  89. meta = util.MaybeGzipData(meta)
  90. }
  91. if err := store.session.Query(
  92. "INSERT INTO filemeta (directory,name,meta) VALUES(?,?,?) USING TTL ? ",
  93. dir, name, meta, entry.TtlSec).Exec(); err != nil {
  94. return fmt.Errorf("insert %s: %s", entry.FullPath, err)
  95. }
  96. return nil
  97. }
  98. func (store *CassandraStore) UpdateEntry(ctx context.Context, entry *filer.Entry) (err error) {
  99. return store.InsertEntry(ctx, entry)
  100. }
  101. func (store *CassandraStore) FindEntry(ctx context.Context, fullpath util.FullPath) (entry *filer.Entry, err error) {
  102. dir, name := fullpath.DirAndName()
  103. if dirHash, ok := store.isSuperLargeDirectory(dir); ok {
  104. dir, name = dirHash+name, ""
  105. }
  106. var data []byte
  107. if err := store.session.Query(
  108. "SELECT meta FROM filemeta WHERE directory=? AND name=?",
  109. dir, name).Scan(&data); err != nil {
  110. if err != gocql.ErrNotFound {
  111. return nil, filer_pb.ErrNotFound
  112. }
  113. }
  114. if len(data) == 0 {
  115. return nil, filer_pb.ErrNotFound
  116. }
  117. entry = &filer.Entry{
  118. FullPath: fullpath,
  119. }
  120. err = entry.DecodeAttributesAndChunks(util.MaybeDecompressData(data))
  121. if err != nil {
  122. return entry, fmt.Errorf("decode %s : %v", entry.FullPath, err)
  123. }
  124. return entry, nil
  125. }
  126. func (store *CassandraStore) DeleteEntry(ctx context.Context, fullpath util.FullPath) error {
  127. dir, name := fullpath.DirAndName()
  128. if dirHash, ok := store.isSuperLargeDirectory(dir); ok {
  129. dir, name = dirHash+name, ""
  130. }
  131. if err := store.session.Query(
  132. "DELETE FROM filemeta WHERE directory=? AND name=?",
  133. dir, name).Exec(); err != nil {
  134. return fmt.Errorf("delete %s : %v", fullpath, err)
  135. }
  136. return nil
  137. }
  138. func (store *CassandraStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath) error {
  139. if _, ok := store.isSuperLargeDirectory(string(fullpath)); ok {
  140. return nil // filer.ErrUnsupportedSuperLargeDirectoryListing
  141. }
  142. if err := store.session.Query(
  143. "DELETE FROM filemeta WHERE directory=?",
  144. fullpath).Exec(); err != nil {
  145. return fmt.Errorf("delete %s : %v", fullpath, err)
  146. }
  147. return nil
  148. }
  149. func (store *CassandraStore) ListDirectoryPrefixedEntries(ctx context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, prefix string, eachEntryFunc filer.ListEachEntryFunc) (lastFileName string, err error) {
  150. return lastFileName, filer.ErrUnsupportedListDirectoryPrefixed
  151. }
  152. func (store *CassandraStore) ListDirectoryEntries(ctx context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, eachEntryFunc filer.ListEachEntryFunc) (lastFileName string, err error) {
  153. if _, ok := store.isSuperLargeDirectory(string(dirPath)); ok {
  154. return // nil, filer.ErrUnsupportedSuperLargeDirectoryListing
  155. }
  156. cqlStr := "SELECT NAME, meta FROM filemeta WHERE directory=? AND name>? ORDER BY NAME ASC LIMIT ?"
  157. if includeStartFile {
  158. cqlStr = "SELECT NAME, meta FROM filemeta WHERE directory=? AND name>=? ORDER BY NAME ASC LIMIT ?"
  159. }
  160. var data []byte
  161. var name string
  162. iter := store.session.Query(cqlStr, string(dirPath), startFileName, limit+1).Iter()
  163. for iter.Scan(&name, &data) {
  164. entry := &filer.Entry{
  165. FullPath: util.NewFullPath(string(dirPath), name),
  166. }
  167. lastFileName = name
  168. if decodeErr := entry.DecodeAttributesAndChunks(util.MaybeDecompressData(data)); decodeErr != nil {
  169. err = decodeErr
  170. glog.V(0).Infof("list %s : %v", entry.FullPath, err)
  171. break
  172. }
  173. if !eachEntryFunc(entry) {
  174. break
  175. }
  176. }
  177. if err := iter.Close(); err != nil {
  178. glog.V(0).Infof("list iterator close: %v", err)
  179. }
  180. return lastFileName, err
  181. }
  182. func (store *CassandraStore) Shutdown() {
  183. store.session.Close()
  184. }