more cleanup / refactor. almost ready for another review
This commit is contained in:
parent
65d98dfaa4
commit
e2d1fa8558
23 changed files with 1160 additions and 1916 deletions
433
db/db.go
433
db/db.go
|
@ -12,6 +12,7 @@ import (
|
|||
"github.com/lbryio/hub/db/prefixes"
|
||||
"github.com/lbryio/hub/internal"
|
||||
"github.com/lbryio/hub/internal/metrics"
|
||||
pb "github.com/lbryio/hub/protobuf/go"
|
||||
"github.com/linxGnu/grocksdb"
|
||||
|
||||
log "github.com/sirupsen/logrus"
|
||||
|
@ -153,92 +154,128 @@ func (x *ExpandedResolveResult) String() string {
|
|||
return fmt.Sprintf("ExpandedResolveResult{Stream: %s, Channel: %s, Repost: %s, RepostedChannel: %s}", x.Stream, x.Channel, x.Repost, x.RepostedChannel)
|
||||
}
|
||||
|
||||
type IterOptions struct {
|
||||
FillCache bool
|
||||
Prefix []byte
|
||||
Start []byte //interface{}
|
||||
Stop []byte //interface{}
|
||||
IncludeStart bool
|
||||
IncludeStop bool
|
||||
IncludeKey bool
|
||||
IncludeValue bool
|
||||
RawKey bool
|
||||
RawValue bool
|
||||
CfHandle *grocksdb.ColumnFamilyHandle
|
||||
It *grocksdb.Iterator
|
||||
}
|
||||
|
||||
// NewIterateOptions creates a defualt options structure for a db iterator.
|
||||
func NewIterateOptions() *IterOptions {
|
||||
return &IterOptions{
|
||||
FillCache: false,
|
||||
Prefix: []byte{},
|
||||
Start: nil,
|
||||
Stop: nil,
|
||||
IncludeStart: true,
|
||||
IncludeStop: false,
|
||||
IncludeKey: true,
|
||||
IncludeValue: false,
|
||||
RawKey: false,
|
||||
RawValue: false,
|
||||
CfHandle: nil,
|
||||
It: nil,
|
||||
func (res *ExpandedResolveResult) ToOutputs() ([]*pb.Output, []*pb.Output, error) {
|
||||
txos := make([]*pb.Output, 0)
|
||||
extraTxos := make([]*pb.Output, 0)
|
||||
// Errors
|
||||
if x := res.Channel.GetError(); x != nil {
|
||||
log.Warn("Channel error: ", x)
|
||||
outputErr := &pb.Output_Error{
|
||||
Error: &pb.Error{
|
||||
Text: x.Error.Error(),
|
||||
Code: pb.Error_Code(x.ErrorType),
|
||||
},
|
||||
}
|
||||
res := &pb.Output{Meta: outputErr}
|
||||
txos = append(txos, res)
|
||||
return txos, nil, nil
|
||||
}
|
||||
if x := res.Stream.GetError(); x != nil {
|
||||
log.Warn("Stream error: ", x)
|
||||
outputErr := &pb.Output_Error{
|
||||
Error: &pb.Error{
|
||||
Text: x.Error.Error(),
|
||||
Code: pb.Error_Code(x.ErrorType),
|
||||
},
|
||||
}
|
||||
res := &pb.Output{Meta: outputErr}
|
||||
txos = append(txos, res)
|
||||
return txos, nil, nil
|
||||
}
|
||||
|
||||
// Not errors
|
||||
var channel, stream, repost, repostedChannel *ResolveResult
|
||||
|
||||
channel = res.Channel.GetResult()
|
||||
stream = res.Stream.GetResult()
|
||||
repost = res.Repost.GetResult()
|
||||
repostedChannel = res.RepostedChannel.GetResult()
|
||||
|
||||
if channel != nil && stream == nil {
|
||||
// Channel
|
||||
output := channel.ToOutput()
|
||||
txos = append(txos, output)
|
||||
|
||||
if repost != nil {
|
||||
output := repost.ToOutput()
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
if repostedChannel != nil {
|
||||
output := repostedChannel.ToOutput()
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
|
||||
return txos, extraTxos, nil
|
||||
} else if stream != nil {
|
||||
output := stream.ToOutput()
|
||||
txos = append(txos, output)
|
||||
if channel != nil {
|
||||
output := channel.ToOutput()
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
if repost != nil {
|
||||
output := repost.ToOutput()
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
if repostedChannel != nil {
|
||||
output := repostedChannel.ToOutput()
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
|
||||
return txos, extraTxos, nil
|
||||
}
|
||||
|
||||
return nil, nil, nil
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithCfHandle(cfHandle *grocksdb.ColumnFamilyHandle) *IterOptions {
|
||||
o.CfHandle = cfHandle
|
||||
return o
|
||||
}
|
||||
// ToOutput
|
||||
func (res *ResolveResult) ToOutput() *pb.Output {
|
||||
// func ResolveResultToOutput(res *db.ResolveResult, outputType byte) *OutputWType {
|
||||
// res.ClaimHash
|
||||
var channelOutput *pb.Output
|
||||
var repostOutput *pb.Output
|
||||
|
||||
func (o *IterOptions) WithFillCache(fillCache bool) *IterOptions {
|
||||
o.FillCache = fillCache
|
||||
return o
|
||||
}
|
||||
if res.ChannelTxHash != nil {
|
||||
channelOutput = &pb.Output{
|
||||
TxHash: res.ChannelTxHash,
|
||||
Nout: uint32(res.ChannelTxPostition),
|
||||
Height: res.ChannelHeight,
|
||||
}
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithPrefix(prefix []byte) *IterOptions {
|
||||
o.Prefix = prefix
|
||||
return o
|
||||
}
|
||||
if res.RepostTxHash != nil {
|
||||
repostOutput = &pb.Output{
|
||||
TxHash: res.RepostTxHash,
|
||||
Nout: uint32(res.RepostTxPostition),
|
||||
Height: res.RepostHeight,
|
||||
}
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithStart(start []byte) *IterOptions {
|
||||
o.Start = start
|
||||
return o
|
||||
}
|
||||
claimMeta := &pb.ClaimMeta{
|
||||
Channel: channelOutput,
|
||||
Repost: repostOutput,
|
||||
ShortUrl: res.ShortUrl,
|
||||
Reposted: uint32(res.Reposted),
|
||||
IsControlling: res.IsControlling,
|
||||
CreationHeight: res.CreationHeight,
|
||||
ExpirationHeight: res.ExpirationHeight,
|
||||
ClaimsInChannel: res.ClaimsInChannel,
|
||||
EffectiveAmount: res.EffectiveAmount,
|
||||
SupportAmount: res.SupportAmount,
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithStop(stop []byte) *IterOptions {
|
||||
o.Stop = stop
|
||||
return o
|
||||
}
|
||||
claim := &pb.Output_Claim{
|
||||
Claim: claimMeta,
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeStart(includeStart bool) *IterOptions {
|
||||
o.IncludeStart = includeStart
|
||||
return o
|
||||
}
|
||||
output := &pb.Output{
|
||||
TxHash: res.TxHash,
|
||||
Nout: uint32(res.Position),
|
||||
Height: res.Height,
|
||||
Meta: claim,
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeStop(includeStop bool) *IterOptions {
|
||||
o.IncludeStop = includeStop
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeKey(includeKey bool) *IterOptions {
|
||||
o.IncludeKey = includeKey
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeValue(includeValue bool) *IterOptions {
|
||||
o.IncludeValue = includeValue
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithRawKey(rawKey bool) *IterOptions {
|
||||
o.RawKey = rawKey
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithRawValue(rawValue bool) *IterOptions {
|
||||
o.RawValue = rawValue
|
||||
return o
|
||||
return output
|
||||
}
|
||||
|
||||
type PathSegment struct {
|
||||
|
@ -286,95 +323,6 @@ func intMin(a, b int) int {
|
|||
return b
|
||||
}
|
||||
|
||||
func (o *IterOptions) StopIteration(key []byte) bool {
|
||||
if key == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
// TODO: Look at not doing floating point conversions for this
|
||||
maxLenStop := intMin(len(key), len(o.Stop))
|
||||
maxLenStart := intMin(len(key), len(o.Start))
|
||||
if o.Stop != nil &&
|
||||
(bytes.HasPrefix(key, o.Stop) || bytes.Compare(o.Stop, key[:maxLenStop]) < 0) {
|
||||
return true
|
||||
} else if o.Start != nil &&
|
||||
bytes.Compare(o.Start, key[:maxLenStart]) > 0 {
|
||||
return true
|
||||
} else if o.Prefix != nil && !bytes.HasPrefix(key, o.Prefix) {
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
func (opts *IterOptions) ReadRow(prevKey *[]byte) *prefixes.PrefixRowKV {
|
||||
it := opts.It
|
||||
if !it.Valid() {
|
||||
log.Trace("ReadRow iterator not valid returning nil")
|
||||
return nil
|
||||
}
|
||||
|
||||
key := it.Key()
|
||||
keyData := key.Data()
|
||||
keyLen := len(keyData)
|
||||
value := it.Value()
|
||||
valueData := value.Data()
|
||||
valueLen := len(valueData)
|
||||
|
||||
var outKey interface{} = nil
|
||||
var outValue interface{} = nil
|
||||
var err error = nil
|
||||
|
||||
log.Trace("keyData:", keyData)
|
||||
log.Trace("valueData:", valueData)
|
||||
|
||||
// We need to check the current key if we're not including the stop
|
||||
// key.
|
||||
if !opts.IncludeStop && opts.StopIteration(keyData) {
|
||||
log.Trace("ReadRow returning nil")
|
||||
return nil
|
||||
}
|
||||
|
||||
// We have to copy the key no matter what because we need to check
|
||||
// it on the next iterations to see if we're going to stop.
|
||||
newKeyData := make([]byte, keyLen)
|
||||
copy(newKeyData, keyData)
|
||||
if opts.IncludeKey && !opts.RawKey {
|
||||
outKey, err = prefixes.UnpackGenericKey(newKeyData)
|
||||
if err != nil {
|
||||
log.Error(err)
|
||||
}
|
||||
} else if opts.IncludeKey {
|
||||
outKey = newKeyData
|
||||
}
|
||||
|
||||
// Value could be quite large, so this setting could be important
|
||||
// for performance in some cases.
|
||||
if opts.IncludeValue {
|
||||
newValueData := make([]byte, valueLen)
|
||||
copy(newValueData, valueData)
|
||||
if !opts.RawValue {
|
||||
outValue, err = prefixes.UnpackGenericValue(newKeyData, newValueData)
|
||||
if err != nil {
|
||||
log.Error(err)
|
||||
}
|
||||
} else {
|
||||
outValue = newValueData
|
||||
}
|
||||
}
|
||||
|
||||
key.Free()
|
||||
value.Free()
|
||||
|
||||
kv := &prefixes.PrefixRowKV{
|
||||
Key: outKey,
|
||||
Value: outValue,
|
||||
}
|
||||
*prevKey = newKeyData
|
||||
|
||||
return kv
|
||||
}
|
||||
|
||||
func IterCF(db *grocksdb.DB, opts *IterOptions) <-chan *prefixes.PrefixRowKV {
|
||||
ch := make(chan *prefixes.PrefixRowKV)
|
||||
|
||||
|
@ -501,7 +449,7 @@ func GetProdDB(name string, secondaryPath string) (*ReadOnlyDBColumnFamily, func
|
|||
|
||||
cleanup := func() {
|
||||
db.DB.Close()
|
||||
err = os.RemoveAll(fmt.Sprintf("./%s", secondaryPath))
|
||||
err = os.RemoveAll(secondaryPath)
|
||||
if err != nil {
|
||||
log.Println(err)
|
||||
}
|
||||
|
@ -512,6 +460,17 @@ func GetProdDB(name string, secondaryPath string) (*ReadOnlyDBColumnFamily, func
|
|||
return nil, cleanup, err
|
||||
}
|
||||
|
||||
// Wait for the height to be greater than zero
|
||||
// for {
|
||||
// ReadDBState(db)
|
||||
// if db.LastState.Height > 0 {
|
||||
// logrus.Infof("db height is > 0: %+v\n", db.LastState)
|
||||
// break
|
||||
// }
|
||||
// time.Sleep(time.Millisecond * 100)
|
||||
// logrus.Infof("Waiting for db height to be > 0: %+v\n", db.LastState)
|
||||
// }
|
||||
|
||||
return db, cleanup, nil
|
||||
}
|
||||
|
||||
|
@ -555,22 +514,22 @@ func GetDBColumnFamlies(name string, secondayPath string, cfNames []string) (*Re
|
|||
DoneChan: make(chan struct{}),
|
||||
}
|
||||
|
||||
err = ReadDBState(myDB) //TODO: Figure out right place for this
|
||||
err = myDB.ReadDBState() //TODO: Figure out right place for this
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = InitTxCounts(myDB)
|
||||
err = myDB.InitTxCounts()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = InitHeaders(myDB)
|
||||
err = myDB.InitHeaders()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = GetBlocksAndFilters(myDB)
|
||||
err = myDB.GetBlocksAndFilters()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -578,20 +537,21 @@ func GetDBColumnFamlies(name string, secondayPath string, cfNames []string) (*Re
|
|||
return myDB, nil
|
||||
}
|
||||
|
||||
func Advance(db *ReadOnlyDBColumnFamily, height uint32) {
|
||||
// Advance advance the db to the given height.
|
||||
func (db *ReadOnlyDBColumnFamily) Advance(height uint32) {
|
||||
// TODO: assert tx_count not in self.db.tx_counts, f'boom {tx_count} in {len(self.db.tx_counts)} tx counts'
|
||||
if db.TxCounts.Len() != height {
|
||||
log.Error("tx count len:", db.TxCounts.Len(), "height:", height)
|
||||
return
|
||||
}
|
||||
|
||||
headerObj, err := GetHeader(db, height)
|
||||
headerObj, err := db.GetHeader(height)
|
||||
if err != nil {
|
||||
log.Error("getting header:", err)
|
||||
return
|
||||
}
|
||||
|
||||
txCountObj, err := GetTxCount(db, height)
|
||||
txCountObj, err := db.GetTxCount(height)
|
||||
if err != nil {
|
||||
log.Error("getting tx count:", err)
|
||||
return
|
||||
|
@ -607,12 +567,13 @@ func Advance(db *ReadOnlyDBColumnFamily, height uint32) {
|
|||
db.Headers.Push(headerObj)
|
||||
}
|
||||
|
||||
func Unwind(db *ReadOnlyDBColumnFamily) {
|
||||
// Unwind unwinds the db one block height
|
||||
func (db *ReadOnlyDBColumnFamily) Unwind() {
|
||||
db.TxCounts.Pop()
|
||||
db.Headers.Pop()
|
||||
}
|
||||
|
||||
func Shutdown(db *ReadOnlyDBColumnFamily) {
|
||||
func (db *ReadOnlyDBColumnFamily) Shutdown() {
|
||||
db.ShutdownChan <- struct{}{}
|
||||
<-db.DoneChan
|
||||
db.Cleanup()
|
||||
|
@ -621,13 +582,18 @@ func Shutdown(db *ReadOnlyDBColumnFamily) {
|
|||
// RunDetectChanges Go routine the runs continuously while the hub is active
|
||||
// to keep the db readonly view up to date and handle reorgs on the
|
||||
// blockchain.
|
||||
func RunDetectChanges(db *ReadOnlyDBColumnFamily) {
|
||||
func (db *ReadOnlyDBColumnFamily) RunDetectChanges(notifCh chan *internal.HeightHash) {
|
||||
go func() {
|
||||
lastPrint := time.Now()
|
||||
for {
|
||||
// FIXME: Figure out best sleep interval
|
||||
err := DetectChanges(db)
|
||||
if time.Since(lastPrint) > time.Second {
|
||||
log.Debug("DetectChanges:", db.LastState)
|
||||
lastPrint = time.Now()
|
||||
}
|
||||
err := db.DetectChanges(notifCh)
|
||||
if err != nil {
|
||||
log.Printf("Error detecting changes: %#v\n", err)
|
||||
log.Infof("Error detecting changes: %#v", err)
|
||||
}
|
||||
select {
|
||||
case <-db.ShutdownChan:
|
||||
|
@ -640,13 +606,13 @@ func RunDetectChanges(db *ReadOnlyDBColumnFamily) {
|
|||
}
|
||||
|
||||
// DetectChanges keep the rocksdb db in sync and handle reorgs
|
||||
func DetectChanges(db *ReadOnlyDBColumnFamily) error {
|
||||
func (db *ReadOnlyDBColumnFamily) DetectChanges(notifCh chan *internal.HeightHash) error {
|
||||
err := db.DB.TryCatchUpWithPrimary()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
state, err := GetDBState(db)
|
||||
state, err := db.GetDBState()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -667,11 +633,15 @@ func DetectChanges(db *ReadOnlyDBColumnFamily) error {
|
|||
if db.LastState != nil {
|
||||
lastHeight = db.LastState.Height
|
||||
for {
|
||||
lastHeightHeader, err := GetHeader(db, lastHeight)
|
||||
lastHeightHeader, err := db.GetHeader(lastHeight)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
curHeader := db.Headers.GetTip().([]byte)
|
||||
curHeaderObj := db.Headers.GetTip()
|
||||
if curHeaderObj == nil {
|
||||
break
|
||||
}
|
||||
curHeader := curHeaderObj.([]byte)
|
||||
log.Debugln("lastHeightHeader: ", hex.EncodeToString(lastHeightHeader))
|
||||
log.Debugln("curHeader: ", hex.EncodeToString(curHeader))
|
||||
if bytes.Equal(curHeader, lastHeightHeader) {
|
||||
|
@ -679,7 +649,7 @@ func DetectChanges(db *ReadOnlyDBColumnFamily) error {
|
|||
break
|
||||
} else {
|
||||
log.Infoln("disconnect block", lastHeight)
|
||||
Unwind(db)
|
||||
db.Unwind()
|
||||
rewound = true
|
||||
lastHeight -= 1
|
||||
time.Sleep(time.Second)
|
||||
|
@ -688,9 +658,14 @@ func DetectChanges(db *ReadOnlyDBColumnFamily) error {
|
|||
}
|
||||
if rewound {
|
||||
metrics.ReorgCount.Inc()
|
||||
hash, err := db.GetBlockHash(lastHeight)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
notifCh <- &internal.HeightHash{Height: uint64(lastHeight), BlockHash: hash}
|
||||
}
|
||||
|
||||
err = ReadDBState(db)
|
||||
err = db.ReadDBState()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -698,7 +673,13 @@ func DetectChanges(db *ReadOnlyDBColumnFamily) error {
|
|||
if db.LastState == nil || lastHeight < state.Height {
|
||||
for height := lastHeight + 1; height <= state.Height; height++ {
|
||||
log.Info("advancing to: ", height)
|
||||
Advance(db, height)
|
||||
db.Advance(height)
|
||||
hash, err := db.GetBlockHash(height)
|
||||
if err != nil {
|
||||
log.Info("error getting block hash: ", err)
|
||||
return err
|
||||
}
|
||||
notifCh <- &internal.HeightHash{Height: uint64(height), BlockHash: hash}
|
||||
}
|
||||
//TODO: ClearCache
|
||||
log.Warn("implement cache clearing")
|
||||
|
@ -713,58 +694,10 @@ func DetectChanges(db *ReadOnlyDBColumnFamily) error {
|
|||
}
|
||||
|
||||
return nil
|
||||
/*
|
||||
self.db.blocked_streams, self.db.blocked_channels = self.db.get_streams_and_channels_reposted_by_channel_hashes(
|
||||
self.db.blocking_channel_hashes
|
||||
)
|
||||
self.db.filtered_streams, self.db.filtered_channels = self.db.get_streams_and_channels_reposted_by_channel_hashes(
|
||||
self.db.filtering_channel_hashes
|
||||
)
|
||||
*/
|
||||
}
|
||||
|
||||
/*
|
||||
def read_db_state(self):
|
||||
state = self.prefix_db.db_state.get()
|
||||
|
||||
if not state:
|
||||
self.db_height = -1
|
||||
self.db_tx_count = 0
|
||||
self.db_tip = b'\0' * 32
|
||||
self.db_version = max(self.DB_VERSIONS)
|
||||
self.utxo_flush_count = 0
|
||||
self.wall_time = 0
|
||||
self.first_sync = True
|
||||
self.hist_flush_count = 0
|
||||
self.hist_comp_flush_count = -1
|
||||
self.hist_comp_cursor = -1
|
||||
self.hist_db_version = max(self.DB_VERSIONS)
|
||||
self.es_sync_height = 0
|
||||
else:
|
||||
self.db_version = state.db_version
|
||||
if self.db_version not in self.DB_VERSIONS:
|
||||
raise DBError(f'your DB version is {self.db_version} but this '
|
||||
f'software only handles versions {self.DB_VERSIONS}')
|
||||
# backwards compat
|
||||
genesis_hash = state.genesis
|
||||
if genesis_hash.hex() != self.coin.GENESIS_HASH:
|
||||
raise DBError(f'DB genesis hash {genesis_hash} does not '
|
||||
f'match coin {self.coin.GENESIS_HASH}')
|
||||
self.db_height = state.height
|
||||
self.db_tx_count = state.tx_count
|
||||
self.db_tip = state.tip
|
||||
self.utxo_flush_count = state.utxo_flush_count
|
||||
self.wall_time = state.wall_time
|
||||
self.first_sync = state.first_sync
|
||||
self.hist_flush_count = state.hist_flush_count
|
||||
self.hist_comp_flush_count = state.comp_flush_count
|
||||
self.hist_comp_cursor = state.comp_cursor
|
||||
self.hist_db_version = state.db_version
|
||||
self.es_sync_height = state.es_sync_height
|
||||
return state
|
||||
*/
|
||||
func ReadDBState(db *ReadOnlyDBColumnFamily) error {
|
||||
state, err := GetDBState(db)
|
||||
func (db *ReadOnlyDBColumnFamily) ReadDBState() error {
|
||||
state, err := db.GetDBState()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -777,8 +710,8 @@ func ReadDBState(db *ReadOnlyDBColumnFamily) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func InitHeaders(db *ReadOnlyDBColumnFamily) error {
|
||||
handle, err := EnsureHandle(db, prefixes.Header)
|
||||
func (db *ReadOnlyDBColumnFamily) InitHeaders() error {
|
||||
handle, err := db.EnsureHandle(prefixes.Header)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -787,12 +720,12 @@ func InitHeaders(db *ReadOnlyDBColumnFamily) error {
|
|||
db.Headers = db_stack.NewSliceBackedStack(12000)
|
||||
|
||||
startKey := prefixes.NewHeaderKey(0)
|
||||
endKey := prefixes.NewHeaderKey(db.LastState.Height)
|
||||
// endKey := prefixes.NewHeaderKey(db.LastState.Height)
|
||||
startKeyRaw := startKey.PackKey()
|
||||
endKeyRaw := endKey.PackKey()
|
||||
// endKeyRaw := endKey.PackKey()
|
||||
options := NewIterateOptions().WithPrefix([]byte{prefixes.Header}).WithCfHandle(handle)
|
||||
options = options.WithIncludeKey(false).WithIncludeValue(true).WithIncludeStop(true)
|
||||
options = options.WithStart(startKeyRaw).WithStop(endKeyRaw)
|
||||
options = options.WithIncludeKey(false).WithIncludeValue(true) //.WithIncludeStop(true)
|
||||
options = options.WithStart(startKeyRaw) //.WithStop(endKeyRaw)
|
||||
|
||||
ch := IterCF(db.DB, options)
|
||||
|
||||
|
@ -804,9 +737,9 @@ func InitHeaders(db *ReadOnlyDBColumnFamily) error {
|
|||
}
|
||||
|
||||
// InitTxCounts initializes the txCounts map
|
||||
func InitTxCounts(db *ReadOnlyDBColumnFamily) error {
|
||||
func (db *ReadOnlyDBColumnFamily) InitTxCounts() error {
|
||||
start := time.Now()
|
||||
handle, err := EnsureHandle(db, prefixes.TxCount)
|
||||
handle, err := db.EnsureHandle(prefixes.TxCount)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -838,13 +771,13 @@ func InitTxCounts(db *ReadOnlyDBColumnFamily) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// RunGetBlocksAndFiltes Go routine that runs continuously while the hub is active
|
||||
// RunGetBlocksAndFilters Go routine that runs continuously while the hub is active
|
||||
// to keep the blocked and filtered channels and streams up to date.
|
||||
func RunGetBlocksAndFiltes(db *ReadOnlyDBColumnFamily) {
|
||||
func (db *ReadOnlyDBColumnFamily) RunGetBlocksAndFilters() {
|
||||
go func() {
|
||||
for {
|
||||
// FIXME: Figure out best sleep interval
|
||||
err := GetBlocksAndFilters(db)
|
||||
err := db.GetBlocksAndFilters()
|
||||
if err != nil {
|
||||
log.Printf("Error getting blocked and filtered chanels: %#v\n", err)
|
||||
}
|
||||
|
@ -853,8 +786,8 @@ func RunGetBlocksAndFiltes(db *ReadOnlyDBColumnFamily) {
|
|||
}()
|
||||
}
|
||||
|
||||
func GetBlocksAndFilters(db *ReadOnlyDBColumnFamily) error {
|
||||
blockedChannels, blockedStreams, err := GetStreamsAndChannelRepostedByChannelHashes(db, db.BlockingChannelHashes)
|
||||
func (db *ReadOnlyDBColumnFamily) GetBlocksAndFilters() error {
|
||||
blockedChannels, blockedStreams, err := db.GetStreamsAndChannelRepostedByChannelHashes(db.BlockingChannelHashes)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -862,7 +795,7 @@ func GetBlocksAndFilters(db *ReadOnlyDBColumnFamily) error {
|
|||
db.BlockedChannels = blockedChannels
|
||||
db.BlockedStreams = blockedStreams
|
||||
|
||||
filteredChannels, filteredStreams, err := GetStreamsAndChannelRepostedByChannelHashes(db, db.FilteringChannelHashes)
|
||||
filteredChannels, filteredStreams, err := db.GetStreamsAndChannelRepostedByChannelHashes(db.FilteringChannelHashes)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
150
db/db_get.go
150
db/db_get.go
|
@ -24,7 +24,7 @@ func GetExpirationHeightFull(lastUpdatedHeight uint32, extended bool) uint32 {
|
|||
}
|
||||
|
||||
// EnsureHandle is a helper function to ensure that the db has a handle to the given column family.
|
||||
func EnsureHandle(db *ReadOnlyDBColumnFamily, prefix byte) (*grocksdb.ColumnFamilyHandle, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) EnsureHandle(prefix byte) (*grocksdb.ColumnFamilyHandle, error) {
|
||||
cfName := string(prefix)
|
||||
handle := db.Handles[cfName]
|
||||
if handle == nil {
|
||||
|
@ -33,8 +33,28 @@ func EnsureHandle(db *ReadOnlyDBColumnFamily, prefix byte) (*grocksdb.ColumnFami
|
|||
return handle, nil
|
||||
}
|
||||
|
||||
func GetHeader(db *ReadOnlyDBColumnFamily, height uint32) ([]byte, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.Header)
|
||||
func (db *ReadOnlyDBColumnFamily) GetBlockHash(height uint32) ([]byte, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.BlockHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
key := prefixes.NewBlockHashKey(height)
|
||||
rawKey := key.PackKey()
|
||||
slice, err := db.DB.GetCF(db.Opts, handle, rawKey)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
} else if slice.Size() == 0 {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rawValue := make([]byte, len(slice.Data()))
|
||||
copy(rawValue, slice.Data())
|
||||
return rawValue, nil
|
||||
}
|
||||
|
||||
func (db *ReadOnlyDBColumnFamily) GetHeader(height uint32) ([]byte, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.Header)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -53,34 +73,8 @@ func GetHeader(db *ReadOnlyDBColumnFamily, height uint32) ([]byte, error) {
|
|||
return rawValue, nil
|
||||
}
|
||||
|
||||
/*
|
||||
async def reload_blocking_filtering_streams(self):
|
||||
def reload():
|
||||
self.blocked_streams, self.blocked_channels = self.get_streams_and_channels_reposted_by_channel_hashes(
|
||||
self.blocking_channel_hashes
|
||||
)
|
||||
self.filtered_streams, self.filtered_channels = self.get_streams_and_channels_reposted_by_channel_hashes(
|
||||
self.filtering_channel_hashes
|
||||
)
|
||||
await asyncio.get_event_loop().run_in_executor(self._executor, reload)
|
||||
|
||||
def get_streams_and_channels_reposted_by_channel_hashes(self, reposter_channel_hashes: Set[bytes]):
|
||||
streams, channels = {}, {}
|
||||
for reposter_channel_hash in reposter_channel_hashes:
|
||||
for stream in self.prefix_db.channel_to_claim.iterate((reposter_channel_hash, ), include_key=False):
|
||||
repost = self.get_repost(stream.claim_hash)
|
||||
if repost:
|
||||
txo = self.get_claim_txo(repost)
|
||||
if txo:
|
||||
if txo.normalized_name.startswith('@'):
|
||||
channels[repost] = reposter_channel_hash
|
||||
else:
|
||||
streams[repost] = reposter_channel_hash
|
||||
return streams, channels
|
||||
*/
|
||||
|
||||
func GetStreamsAndChannelRepostedByChannelHashes(db *ReadOnlyDBColumnFamily, reposterChannelHashes [][]byte) (map[string][]byte, map[string][]byte, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.ChannelToClaim)
|
||||
func (db *ReadOnlyDBColumnFamily) GetStreamsAndChannelRepostedByChannelHashes(reposterChannelHashes [][]byte) (map[string][]byte, map[string][]byte, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.ChannelToClaim)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
@ -97,12 +91,12 @@ func GetStreamsAndChannelRepostedByChannelHashes(db *ReadOnlyDBColumnFamily, rep
|
|||
// for stream := range Iterate(db.DB, prefixes.ChannelToClaim, []byte{reposterChannelHash}, false) {
|
||||
for stream := range ch {
|
||||
value := stream.Value.(*prefixes.ChannelToClaimValue)
|
||||
repost, err := GetRepost(db, value.ClaimHash)
|
||||
repost, err := db.GetRepost(value.ClaimHash)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
if repost != nil {
|
||||
txo, err := GetClaimTxo(db, repost)
|
||||
txo, err := db.GetClaimTxo(repost)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
@ -121,8 +115,8 @@ func GetStreamsAndChannelRepostedByChannelHashes(db *ReadOnlyDBColumnFamily, rep
|
|||
return streams, channels, nil
|
||||
}
|
||||
|
||||
func GetClaimsInChannelCount(db *ReadOnlyDBColumnFamily, channelHash []byte) (uint32, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.ChannelCount)
|
||||
func (db *ReadOnlyDBColumnFamily) GetClaimsInChannelCount(channelHash []byte) (uint32, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.ChannelCount)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -144,9 +138,9 @@ func GetClaimsInChannelCount(db *ReadOnlyDBColumnFamily, channelHash []byte) (ui
|
|||
return value.Count, nil
|
||||
}
|
||||
|
||||
func GetShortClaimIdUrl(db *ReadOnlyDBColumnFamily, name string, normalizedName string, claimHash []byte, rootTxNum uint32, rootPosition uint16) (string, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) GetShortClaimIdUrl(name string, normalizedName string, claimHash []byte, rootTxNum uint32, rootPosition uint16) (string, error) {
|
||||
prefix := []byte{prefixes.ClaimShortIdPrefix}
|
||||
handle, err := EnsureHandle(db, prefixes.ClaimShortIdPrefix)
|
||||
handle, err := db.EnsureHandle(prefixes.ClaimShortIdPrefix)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
@ -183,8 +177,8 @@ func GetShortClaimIdUrl(db *ReadOnlyDBColumnFamily, name string, normalizedName
|
|||
return "", nil
|
||||
}
|
||||
|
||||
func GetRepost(db *ReadOnlyDBColumnFamily, claimHash []byte) ([]byte, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.Repost)
|
||||
func (db *ReadOnlyDBColumnFamily) GetRepost(claimHash []byte) ([]byte, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.Repost)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -204,8 +198,8 @@ func GetRepost(db *ReadOnlyDBColumnFamily, claimHash []byte) ([]byte, error) {
|
|||
return value.RepostedClaimHash, nil
|
||||
}
|
||||
|
||||
func GetRepostedCount(db *ReadOnlyDBColumnFamily, claimHash []byte) (int, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.RepostedClaim)
|
||||
func (db *ReadOnlyDBColumnFamily) GetRepostedCount(claimHash []byte) (int, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.RepostedClaim)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -229,8 +223,8 @@ func GetRepostedCount(db *ReadOnlyDBColumnFamily, claimHash []byte) (int, error)
|
|||
return i, nil
|
||||
}
|
||||
|
||||
func GetChannelForClaim(db *ReadOnlyDBColumnFamily, claimHash []byte, txNum uint32, position uint16) ([]byte, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.ClaimToChannel)
|
||||
func (db *ReadOnlyDBColumnFamily) GetChannelForClaim(claimHash []byte, txNum uint32, position uint16) ([]byte, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.ClaimToChannel)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -250,8 +244,8 @@ func GetChannelForClaim(db *ReadOnlyDBColumnFamily, claimHash []byte, txNum uint
|
|||
return value.SigningHash, nil
|
||||
}
|
||||
|
||||
func GetActiveAmount(db *ReadOnlyDBColumnFamily, claimHash []byte, txoType uint8, height uint32) (uint64, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.ActiveAmount)
|
||||
func (db *ReadOnlyDBColumnFamily) GetActiveAmount(claimHash []byte, txoType uint8, height uint32) (uint64, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.ActiveAmount)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -277,8 +271,8 @@ func GetActiveAmount(db *ReadOnlyDBColumnFamily, claimHash []byte, txoType uint8
|
|||
return sum, nil
|
||||
}
|
||||
|
||||
func GetEffectiveAmount(db *ReadOnlyDBColumnFamily, claimHash []byte, supportOnly bool) (uint64, error) {
|
||||
supportAmount, err := GetActiveAmount(db, claimHash, prefixes.ACTIVATED_SUPPORT_TXO_TYPE, db.Height+1)
|
||||
func (db *ReadOnlyDBColumnFamily) GetEffectiveAmount(claimHash []byte, supportOnly bool) (uint64, error) {
|
||||
supportAmount, err := db.GetActiveAmount(claimHash, prefixes.ACTIVATED_SUPPORT_TXO_TYPE, db.Height+1)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -287,7 +281,7 @@ func GetEffectiveAmount(db *ReadOnlyDBColumnFamily, claimHash []byte, supportOnl
|
|||
return supportAmount, nil
|
||||
}
|
||||
|
||||
activationAmount, err := GetActiveAmount(db, claimHash, prefixes.ACTIVATED_CLAIM_TXO_TYPE, db.Height+1)
|
||||
activationAmount, err := db.GetActiveAmount(claimHash, prefixes.ACTIVATED_CLAIM_TXO_TYPE, db.Height+1)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -295,8 +289,8 @@ func GetEffectiveAmount(db *ReadOnlyDBColumnFamily, claimHash []byte, supportOnl
|
|||
return activationAmount + supportAmount, nil
|
||||
}
|
||||
|
||||
func GetSupportAmount(db *ReadOnlyDBColumnFamily, claimHash []byte) (uint64, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.SupportAmount)
|
||||
func (db *ReadOnlyDBColumnFamily) GetSupportAmount(claimHash []byte) (uint64, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.SupportAmount)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -316,14 +310,14 @@ func GetSupportAmount(db *ReadOnlyDBColumnFamily, claimHash []byte) (uint64, err
|
|||
return value.Amount, nil
|
||||
}
|
||||
|
||||
func GetTxHash(db *ReadOnlyDBColumnFamily, txNum uint32) ([]byte, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) GetTxHash(txNum uint32) ([]byte, error) {
|
||||
/*
|
||||
if self._cache_all_tx_hashes:
|
||||
return self.total_transactions[tx_num]
|
||||
return self.prefix_db.tx_hash.get(tx_num, deserialize_value=False)
|
||||
*/
|
||||
// TODO: caching
|
||||
handle, err := EnsureHandle(db, prefixes.TxHash)
|
||||
handle, err := db.EnsureHandle(prefixes.TxHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -343,14 +337,14 @@ func GetTxHash(db *ReadOnlyDBColumnFamily, txNum uint32) ([]byte, error) {
|
|||
return rawValue, nil
|
||||
}
|
||||
|
||||
func GetActivation(db *ReadOnlyDBColumnFamily, txNum uint32, postition uint16) (uint32, error) {
|
||||
return GetActivationFull(db, txNum, postition, false)
|
||||
func (db *ReadOnlyDBColumnFamily) GetActivation(txNum uint32, postition uint16) (uint32, error) {
|
||||
return db.GetActivationFull(txNum, postition, false)
|
||||
}
|
||||
|
||||
func GetActivationFull(db *ReadOnlyDBColumnFamily, txNum uint32, postition uint16, isSupport bool) (uint32, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) GetActivationFull(txNum uint32, postition uint16, isSupport bool) (uint32, error) {
|
||||
var typ uint8
|
||||
|
||||
handle, err := EnsureHandle(db, prefixes.ActivatedClaimAndSupport)
|
||||
handle, err := db.EnsureHandle(prefixes.ActivatedClaimAndSupport)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -374,13 +368,13 @@ func GetActivationFull(db *ReadOnlyDBColumnFamily, txNum uint32, postition uint1
|
|||
return value.Height, nil
|
||||
}
|
||||
|
||||
func GetClaimTxo(db *ReadOnlyDBColumnFamily, claim []byte) (*prefixes.ClaimToTXOValue, error) {
|
||||
return GetCachedClaimTxo(db, claim, false)
|
||||
func (db *ReadOnlyDBColumnFamily) GetClaimTxo(claim []byte) (*prefixes.ClaimToTXOValue, error) {
|
||||
return db.GetCachedClaimTxo(claim, false)
|
||||
}
|
||||
|
||||
func GetCachedClaimTxo(db *ReadOnlyDBColumnFamily, claim []byte, useCache bool) (*prefixes.ClaimToTXOValue, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) GetCachedClaimTxo(claim []byte, useCache bool) (*prefixes.ClaimToTXOValue, error) {
|
||||
// TODO: implement cache
|
||||
handle, err := EnsureHandle(db, prefixes.ClaimToTXO)
|
||||
handle, err := db.EnsureHandle(prefixes.ClaimToTXO)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -401,8 +395,8 @@ func GetCachedClaimTxo(db *ReadOnlyDBColumnFamily, claim []byte, useCache bool)
|
|||
return value, nil
|
||||
}
|
||||
|
||||
func ControllingClaimIter(db *ReadOnlyDBColumnFamily) <-chan *prefixes.PrefixRowKV {
|
||||
handle, err := EnsureHandle(db, prefixes.ClaimTakeover)
|
||||
func (db *ReadOnlyDBColumnFamily) ControllingClaimIter() <-chan *prefixes.PrefixRowKV {
|
||||
handle, err := db.EnsureHandle(prefixes.ClaimTakeover)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
|
@ -416,8 +410,8 @@ func ControllingClaimIter(db *ReadOnlyDBColumnFamily) <-chan *prefixes.PrefixRow
|
|||
return ch
|
||||
}
|
||||
|
||||
func GetControllingClaim(db *ReadOnlyDBColumnFamily, name string) (*prefixes.ClaimTakeoverValue, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.ClaimTakeover)
|
||||
func (db *ReadOnlyDBColumnFamily) GetControllingClaim(name string) (*prefixes.ClaimTakeoverValue, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.ClaimTakeover)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -443,13 +437,13 @@ func GetControllingClaim(db *ReadOnlyDBColumnFamily, name string) (*prefixes.Cla
|
|||
return value, nil
|
||||
}
|
||||
|
||||
func FsGetClaimByHash(db *ReadOnlyDBColumnFamily, claimHash []byte) (*ResolveResult, error) {
|
||||
claim, err := GetCachedClaimTxo(db, claimHash, true)
|
||||
func (db *ReadOnlyDBColumnFamily) FsGetClaimByHash(claimHash []byte) (*ResolveResult, error) {
|
||||
claim, err := db.GetCachedClaimTxo(claimHash, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
activation, err := GetActivation(db, claim.TxNum, claim.Position)
|
||||
activation, err := db.GetActivation(claim.TxNum, claim.Position)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -468,8 +462,8 @@ func FsGetClaimByHash(db *ReadOnlyDBColumnFamily, claimHash []byte) (*ResolveRes
|
|||
)
|
||||
}
|
||||
|
||||
func GetTxCount(db *ReadOnlyDBColumnFamily, height uint32) (*prefixes.TxCountValue, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.TxCount)
|
||||
func (db *ReadOnlyDBColumnFamily) GetTxCount(height uint32) (*prefixes.TxCountValue, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.TxCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -490,8 +484,8 @@ func GetTxCount(db *ReadOnlyDBColumnFamily, height uint32) (*prefixes.TxCountVal
|
|||
return value, nil
|
||||
}
|
||||
|
||||
func GetDBState(db *ReadOnlyDBColumnFamily) (*prefixes.DBStateValue, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.DBState)
|
||||
func (db *ReadOnlyDBColumnFamily) GetDBState() (*prefixes.DBStateValue, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.DBState)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -511,8 +505,8 @@ func GetDBState(db *ReadOnlyDBColumnFamily) (*prefixes.DBStateValue, error) {
|
|||
return value, nil
|
||||
}
|
||||
|
||||
func EffectiveAmountNameIter(db *ReadOnlyDBColumnFamily, normalizedName string) <-chan *prefixes.PrefixRowKV {
|
||||
handle, err := EnsureHandle(db, prefixes.EffectiveAmount)
|
||||
func (db *ReadOnlyDBColumnFamily) EffectiveAmountNameIter(normalizedName string) <-chan *prefixes.PrefixRowKV {
|
||||
handle, err := db.EnsureHandle(prefixes.EffectiveAmount)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
|
@ -526,8 +520,8 @@ func EffectiveAmountNameIter(db *ReadOnlyDBColumnFamily, normalizedName string)
|
|||
return ch
|
||||
}
|
||||
|
||||
func ClaimShortIdIter(db *ReadOnlyDBColumnFamily, normalizedName string, claimId string) <-chan *prefixes.PrefixRowKV {
|
||||
handle, err := EnsureHandle(db, prefixes.ClaimShortIdPrefix)
|
||||
func (db *ReadOnlyDBColumnFamily) ClaimShortIdIter(normalizedName string, claimId string) <-chan *prefixes.PrefixRowKV {
|
||||
handle, err := db.EnsureHandle(prefixes.ClaimShortIdPrefix)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
|
@ -544,9 +538,9 @@ func ClaimShortIdIter(db *ReadOnlyDBColumnFamily, normalizedName string, claimId
|
|||
return ch
|
||||
}
|
||||
|
||||
func GetCachedClaimHash(db *ReadOnlyDBColumnFamily, txNum uint32, position uint16) (*prefixes.TXOToClaimValue, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) GetCachedClaimHash(txNum uint32, position uint16) (*prefixes.TXOToClaimValue, error) {
|
||||
// TODO: implement cache
|
||||
handle, err := EnsureHandle(db, prefixes.TXOToClaim)
|
||||
handle, err := db.EnsureHandle(prefixes.TXOToClaim)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -570,7 +564,7 @@ func GetCachedClaimHash(db *ReadOnlyDBColumnFamily, txNum uint32, position uint1
|
|||
// GetBlockerHash get the hash of the blocker or filterer of the claim.
|
||||
// TODO: this currently converts the byte arrays to strings, which is not
|
||||
// very efficient. Might want to figure out a better way to do this.
|
||||
func GetBlockerHash(db *ReadOnlyDBColumnFamily, claimHash, repostedClaimHash, channelHash []byte) ([]byte, []byte, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) GetBlockerHash(claimHash, repostedClaimHash, channelHash []byte) ([]byte, []byte, error) {
|
||||
claimHashStr := string(claimHash)
|
||||
respostedClaimHashStr := string(repostedClaimHash)
|
||||
channelHashStr := string(channelHash)
|
||||
|
|
|
@ -28,12 +28,12 @@ func PrepareResolveResult(
|
|||
signatureValid bool) (*ResolveResult, error) {
|
||||
|
||||
normalizedName := internal.NormalizeName(name)
|
||||
controllingClaim, err := GetControllingClaim(db, normalizedName)
|
||||
controllingClaim, err := db.GetControllingClaim(normalizedName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
txHash, err := GetTxHash(db, txNum)
|
||||
txHash, err := db.GetTxHash(txNum)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -43,28 +43,28 @@ func PrepareResolveResult(
|
|||
|
||||
expirationHeight := GetExpirationHeight(height)
|
||||
|
||||
supportAmount, err := GetSupportAmount(db, claimHash)
|
||||
supportAmount, err := db.GetSupportAmount(claimHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
claimToTxo, err := GetCachedClaimTxo(db, claimHash, true)
|
||||
claimToTxo, err := db.GetCachedClaimTxo(claimHash, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
claimAmount := claimToTxo.Amount
|
||||
|
||||
effectiveAmount, err := GetEffectiveAmount(db, claimHash, false)
|
||||
effectiveAmount, err := db.GetEffectiveAmount(claimHash, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
channelHash, err := GetChannelForClaim(db, claimHash, txNum, position)
|
||||
channelHash, err := db.GetChannelForClaim(claimHash, txNum, position)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
repostedClaimHash, err := GetRepost(db, claimHash)
|
||||
repostedClaimHash, err := db.GetRepost(claimHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -74,12 +74,12 @@ func PrepareResolveResult(
|
|||
var repostHeight uint32
|
||||
|
||||
if repostedClaimHash != nil {
|
||||
repostTxo, err := GetCachedClaimTxo(db, repostedClaimHash, true)
|
||||
repostTxo, err := db.GetCachedClaimTxo(repostedClaimHash, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if repostTxo != nil {
|
||||
repostTxHash, err = GetTxHash(db, repostTxo.TxNum)
|
||||
repostTxHash, err = db.GetTxHash(repostTxo.TxNum)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -88,13 +88,13 @@ func PrepareResolveResult(
|
|||
}
|
||||
}
|
||||
|
||||
shortUrl, err := GetShortClaimIdUrl(db, name, normalizedName, claimHash, txNum, rootPosition)
|
||||
shortUrl, err := db.GetShortClaimIdUrl(name, normalizedName, claimHash, txNum, rootPosition)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var canonicalUrl string = shortUrl
|
||||
claimsInChannel, err := GetClaimsInChannelCount(db, claimHash)
|
||||
claimsInChannel, err := db.GetClaimsInChannelCount(claimHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -105,18 +105,17 @@ func PrepareResolveResult(
|
|||
|
||||
if channelHash != nil {
|
||||
// Ignore error because we already have this set if this doesn't work
|
||||
channelVals, _ := GetCachedClaimTxo(db, channelHash, true)
|
||||
channelVals, _ := db.GetCachedClaimTxo(channelHash, true)
|
||||
log.Printf("channelVals: %#v\n", channelVals)
|
||||
if channelVals != nil {
|
||||
channelShortUrl, _ := GetShortClaimIdUrl(
|
||||
db,
|
||||
channelShortUrl, _ := db.GetShortClaimIdUrl(
|
||||
channelVals.Name,
|
||||
channelVals.NormalizedName(),
|
||||
channelHash, channelVals.RootTxNum,
|
||||
channelVals.RootPosition,
|
||||
)
|
||||
canonicalUrl = fmt.Sprintf("%s/%s", channelShortUrl, shortUrl)
|
||||
channelTxHash, err = GetTxHash(db, channelVals.TxNum)
|
||||
channelTxHash, err = db.GetTxHash(channelVals.TxNum)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -125,7 +124,7 @@ func PrepareResolveResult(
|
|||
}
|
||||
}
|
||||
|
||||
reposted, err := GetRepostedCount(db, claimHash)
|
||||
reposted, err := db.GetRepostedCount(claimHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -164,18 +163,18 @@ func PrepareResolveResult(
|
|||
}, nil
|
||||
}
|
||||
|
||||
func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*ResolveResult, error) {
|
||||
func (db *ReadOnlyDBColumnFamily) ResolveParsedUrl(parsed *PathSegment) (*ResolveResult, error) {
|
||||
normalizedName := internal.NormalizeName(parsed.name)
|
||||
if (parsed.amountOrder == -1 && parsed.claimId == "") || parsed.amountOrder == 1 {
|
||||
log.Warn("Resolving claim by name")
|
||||
ch := ControllingClaimIter(db)
|
||||
ch := db.ControllingClaimIter()
|
||||
for kv := range ch {
|
||||
key := kv.Key.(*prefixes.ClaimTakeoverKey)
|
||||
val := kv.Value.(*prefixes.ClaimTakeoverValue)
|
||||
log.Warnf("ClaimTakeoverKey: %#v", key)
|
||||
log.Warnf("ClaimTakeoverValue: %#v", val)
|
||||
}
|
||||
controlling, err := GetControllingClaim(db, normalizedName)
|
||||
controlling, err := db.GetControllingClaim(normalizedName)
|
||||
log.Warnf("controlling: %#v", controlling)
|
||||
log.Warnf("err: %#v", err)
|
||||
if err != nil {
|
||||
|
@ -184,7 +183,7 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
if controlling == nil {
|
||||
return nil, nil
|
||||
}
|
||||
return FsGetClaimByHash(db, controlling.ClaimHash)
|
||||
return db.FsGetClaimByHash(controlling.ClaimHash)
|
||||
}
|
||||
|
||||
var amountOrder int = int(math.Max(float64(parsed.amountOrder), 1))
|
||||
|
@ -200,7 +199,7 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
}
|
||||
|
||||
// Maybe don't use caching version, when I actually implement the cache
|
||||
claimTxo, err := GetCachedClaimTxo(db, claimHash, true)
|
||||
claimTxo, err := db.GetCachedClaimTxo(claimHash, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -209,7 +208,7 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
return nil, nil
|
||||
}
|
||||
|
||||
activation, err := GetActivation(db, claimTxo.TxNum, claimTxo.Position)
|
||||
activation, err := db.GetActivation(claimTxo.TxNum, claimTxo.Position)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -236,7 +235,7 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
j = len(parsed.claimId)
|
||||
}
|
||||
|
||||
ch := ClaimShortIdIter(db, normalizedName, parsed.claimId[:j])
|
||||
ch := db.ClaimShortIdIter(normalizedName, parsed.claimId[:j])
|
||||
row := <-ch
|
||||
if row == nil {
|
||||
return nil, nil
|
||||
|
@ -245,19 +244,19 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
key := row.Key.(*prefixes.ClaimShortIDKey)
|
||||
claimTxo := row.Value.(*prefixes.ClaimShortIDValue)
|
||||
|
||||
fullClaimHash, err := GetCachedClaimHash(db, claimTxo.TxNum, claimTxo.Position)
|
||||
fullClaimHash, err := db.GetCachedClaimHash(claimTxo.TxNum, claimTxo.Position)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
c, err := GetCachedClaimTxo(db, fullClaimHash.ClaimHash, true)
|
||||
c, err := db.GetCachedClaimTxo(fullClaimHash.ClaimHash, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
nonNormalizedName := c.Name
|
||||
signatureIsValid := c.ChannelSignatureIsValid
|
||||
activation, err := GetActivation(db, claimTxo.TxNum, claimTxo.Position)
|
||||
activation, err := db.GetActivation(claimTxo.TxNum, claimTxo.Position)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -279,19 +278,8 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
}
|
||||
|
||||
// Resolve by amount ordering
|
||||
/*
|
||||
for idx, (key, claim_val) in enumerate(self.prefix_db.effective_amount.iterate(prefix=(normalized_name,))):
|
||||
if amount_order > idx + 1:
|
||||
continue
|
||||
claim_txo = self.get_cached_claim_txo(claim_val.claim_hash)
|
||||
activation = self.get_activation(key.tx_num, key.position)
|
||||
return self._prepare_resolve_result(
|
||||
key.tx_num, key.position, claim_val.claim_hash, key.normalized_name, claim_txo.root_tx_num,
|
||||
claim_txo.root_position, activation, claim_txo.channel_signature_is_valid
|
||||
)
|
||||
*/
|
||||
log.Warn("resolving by amount ordering")
|
||||
ch := EffectiveAmountNameIter(db, normalizedName)
|
||||
ch := db.EffectiveAmountNameIter(normalizedName)
|
||||
var i = 0
|
||||
for kv := range ch {
|
||||
if i+1 < amountOrder {
|
||||
|
@ -300,12 +288,12 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
}
|
||||
key := kv.Key.(*prefixes.EffectiveAmountKey)
|
||||
claimVal := kv.Value.(*prefixes.EffectiveAmountValue)
|
||||
claimTxo, err := GetCachedClaimTxo(db, claimVal.ClaimHash, true)
|
||||
claimTxo, err := db.GetCachedClaimTxo(claimVal.ClaimHash, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
activation, err := GetActivation(db, key.TxNum, key.Position)
|
||||
activation, err := db.GetActivation(key.TxNum, key.Position)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -326,8 +314,8 @@ func ResolveParsedUrl(db *ReadOnlyDBColumnFamily, parsed *PathSegment) (*Resolve
|
|||
return nil, nil
|
||||
}
|
||||
|
||||
func ResolveClaimInChannel(db *ReadOnlyDBColumnFamily, channelHash []byte, normalizedName string) (*ResolveResult, error) {
|
||||
handle, err := EnsureHandle(db, prefixes.ChannelToClaim)
|
||||
func (db *ReadOnlyDBColumnFamily) ResolveClaimInChannel(channelHash []byte, normalizedName string) (*ResolveResult, error) {
|
||||
handle, err := db.EnsureHandle(prefixes.ChannelToClaim)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -343,7 +331,7 @@ func ResolveClaimInChannel(db *ReadOnlyDBColumnFamily, channelHash []byte, norma
|
|||
for row := range ch {
|
||||
key := row.Key.(*prefixes.ChannelToClaimKey)
|
||||
stream := row.Value.(*prefixes.ChannelToClaimValue)
|
||||
effectiveAmount, err := GetEffectiveAmount(db, stream.ClaimHash, false)
|
||||
effectiveAmount, err := db.GetEffectiveAmount(stream.ClaimHash, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -376,7 +364,7 @@ func ResolveClaimInChannel(db *ReadOnlyDBColumnFamily, channelHash []byte, norma
|
|||
}
|
||||
}
|
||||
|
||||
func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
||||
func (db *ReadOnlyDBColumnFamily) Resolve(url string) *ExpandedResolveResult {
|
||||
var res = NewExpandedResolveResult()
|
||||
|
||||
var channel *PathSegment = nil
|
||||
|
@ -425,7 +413,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
var resolvedChannel *ResolveResult = nil
|
||||
var resolvedStream *ResolveResult = nil
|
||||
if channel != nil {
|
||||
resolvedChannel, err = ResolveParsedUrl(db, channel)
|
||||
resolvedChannel, err = db.ResolveParsedUrl(channel)
|
||||
if err != nil {
|
||||
res.Channel = &optionalResolveResultOrError{
|
||||
err: &ResolveError{Error: err},
|
||||
|
@ -449,7 +437,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
}
|
||||
if stream != nil {
|
||||
if resolvedChannel != nil {
|
||||
streamClaim, err := ResolveClaimInChannel(db, resolvedChannel.ClaimHash, stream.Normalized())
|
||||
streamClaim, err := db.ResolveClaimInChannel(resolvedChannel.ClaimHash, stream.Normalized())
|
||||
log.Printf("streamClaim %#v\n", streamClaim)
|
||||
if streamClaim != nil {
|
||||
log.Printf("streamClaim.ClaimHash: %s\n", hex.EncodeToString(streamClaim.ClaimHash))
|
||||
|
@ -464,7 +452,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
}
|
||||
|
||||
if streamClaim != nil {
|
||||
resolvedStream, err = FsGetClaimByHash(db, streamClaim.ClaimHash)
|
||||
resolvedStream, err = db.FsGetClaimByHash(streamClaim.ClaimHash)
|
||||
// TODO: Confirm error case
|
||||
if err != nil {
|
||||
res.Stream = &optionalResolveResultOrError{
|
||||
|
@ -474,7 +462,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
resolvedStream, err = ResolveParsedUrl(db, stream)
|
||||
resolvedStream, err = db.ResolveParsedUrl(stream)
|
||||
// TODO: Confirm error case
|
||||
if err != nil {
|
||||
res.Stream = &optionalResolveResultOrError{
|
||||
|
@ -483,7 +471,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
return res
|
||||
}
|
||||
if channel == nil && resolvedChannel == nil && resolvedStream != nil && len(resolvedStream.ChannelHash) > 0 {
|
||||
resolvedChannel, err = FsGetClaimByHash(db, resolvedStream.ChannelHash)
|
||||
resolvedChannel, err = db.FsGetClaimByHash(resolvedStream.ChannelHash)
|
||||
// TODO: Confirm error case
|
||||
if err != nil {
|
||||
res.Channel = &optionalResolveResultOrError{
|
||||
|
@ -524,7 +512,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
claim = resolvedChannel
|
||||
claimHash = resolvedChannel.ClaimHash
|
||||
}
|
||||
blockerHash, _, err = GetBlockerHash(db, claimHash, respostedClaimHash, claim.ChannelHash)
|
||||
blockerHash, _, err = db.GetBlockerHash(claimHash, respostedClaimHash, claim.ChannelHash)
|
||||
log.Printf("blockerHash: %s\n", hex.EncodeToString(blockerHash))
|
||||
if err != nil {
|
||||
res.Channel = &optionalResolveResultOrError{
|
||||
|
@ -533,7 +521,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
return res
|
||||
}
|
||||
if blockerHash != nil {
|
||||
reasonRow, err := FsGetClaimByHash(db, blockerHash)
|
||||
reasonRow, err := db.FsGetClaimByHash(blockerHash)
|
||||
if err != nil {
|
||||
res.Channel = &optionalResolveResultOrError{
|
||||
err: &ResolveError{Error: err},
|
||||
|
@ -546,7 +534,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
return res
|
||||
}
|
||||
if claim.RepostedClaimHash != nil {
|
||||
repost, err = FsGetClaimByHash(db, claim.RepostedClaimHash)
|
||||
repost, err = db.FsGetClaimByHash(claim.RepostedClaimHash)
|
||||
if err != nil {
|
||||
res.Channel = &optionalResolveResultOrError{
|
||||
err: &ResolveError{Error: err},
|
||||
|
@ -554,7 +542,7 @@ func Resolve(db *ReadOnlyDBColumnFamily, url string) *ExpandedResolveResult {
|
|||
return res
|
||||
}
|
||||
if repost != nil && repost.ChannelHash != nil && repost.SignatureValid {
|
||||
repostedChannel, err = FsGetClaimByHash(db, repost.ChannelHash)
|
||||
repostedChannel, err = db.FsGetClaimByHash(repost.ChannelHash)
|
||||
if err != nil {
|
||||
res.Channel = &optionalResolveResultOrError{
|
||||
err: &ResolveError{Error: err},
|
||||
|
|
|
@ -100,7 +100,7 @@ func OpenAndFillTmpDBColumnFamlies(filePath string) (*dbpkg.ReadOnlyDBColumnFami
|
|||
// return nil, nil, nil, err
|
||||
// }
|
||||
|
||||
err = dbpkg.InitTxCounts(myDB)
|
||||
err = myDB.InitTxCounts()
|
||||
if err != nil {
|
||||
return nil, nil, nil, err
|
||||
}
|
||||
|
@ -256,7 +256,7 @@ func TestCatFullDB(t *testing.T) {
|
|||
t.Error(err)
|
||||
return
|
||||
}
|
||||
ch := dbpkg.ClaimShortIdIter(db, "@lbry", "")
|
||||
ch := db.ClaimShortIdIter("@lbry", "")
|
||||
for row := range ch {
|
||||
key := row.Key.(*prefixes.ClaimShortIDKey)
|
||||
val := row.Value.(*prefixes.ClaimShortIDValue)
|
||||
|
@ -287,7 +287,7 @@ func TestOpenFullDB(t *testing.T) {
|
|||
t.Error(err)
|
||||
return
|
||||
}
|
||||
expandedResolveResult := dbpkg.Resolve(db, url)
|
||||
expandedResolveResult := db.Resolve(url)
|
||||
log.Printf("expandedResolveResult: %#v\n", expandedResolveResult)
|
||||
log.Printf("expandedResolveResult: %s\n", expandedResolveResult)
|
||||
}
|
||||
|
@ -302,7 +302,7 @@ func TestResolve(t *testing.T) {
|
|||
return
|
||||
}
|
||||
defer toDefer()
|
||||
expandedResolveResult := dbpkg.Resolve(db, url)
|
||||
expandedResolveResult := db.Resolve(url)
|
||||
log.Printf("%#v\n", expandedResolveResult)
|
||||
if expandedResolveResult != nil && expandedResolveResult.Channel != nil {
|
||||
log.Println(expandedResolveResult.Channel.GetError())
|
||||
|
@ -320,7 +320,7 @@ func TestGetDBState(t *testing.T) {
|
|||
t.Error(err)
|
||||
}
|
||||
defer toDefer()
|
||||
state, err := dbpkg.GetDBState(db)
|
||||
state, err := db.GetDBState()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -342,7 +342,7 @@ func TestGetRepostedClaim(t *testing.T) {
|
|||
}
|
||||
defer toDefer()
|
||||
|
||||
count, err := dbpkg.GetRepostedCount(db, channelHash)
|
||||
count, err := db.GetRepostedCount(channelHash)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -353,7 +353,7 @@ func TestGetRepostedClaim(t *testing.T) {
|
|||
t.Errorf("Expected %d, got %d", want, count)
|
||||
}
|
||||
|
||||
count2, err := dbpkg.GetRepostedCount(db, channelHash2)
|
||||
count2, err := db.GetRepostedCount(channelHash2)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -379,7 +379,7 @@ func TestGetRepost(t *testing.T) {
|
|||
}
|
||||
defer toDefer()
|
||||
|
||||
res, err := dbpkg.GetRepost(db, channelHash)
|
||||
res, err := db.GetRepost(channelHash)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -388,7 +388,7 @@ func TestGetRepost(t *testing.T) {
|
|||
t.Errorf("Expected empty, got %#v", res)
|
||||
}
|
||||
|
||||
res2, err := dbpkg.GetRepost(db, channelHash2)
|
||||
res2, err := db.GetRepost(channelHash2)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -412,7 +412,7 @@ func TestGetClaimsInChannelCount(t *testing.T) {
|
|||
t.Error(err)
|
||||
}
|
||||
defer toDefer()
|
||||
count, err := dbpkg.GetClaimsInChannelCount(db, channelHash)
|
||||
count, err := db.GetClaimsInChannelCount(channelHash)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -441,7 +441,7 @@ func TestGetShortClaimIdUrl(t *testing.T) {
|
|||
t.Error(err)
|
||||
}
|
||||
defer toDefer()
|
||||
shortUrl, err := dbpkg.GetShortClaimIdUrl(db, name, normalName, claimHash, rootTxNum, position)
|
||||
shortUrl, err := db.GetShortClaimIdUrl(name, normalName, claimHash, rootTxNum, position)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -460,7 +460,7 @@ func TestClaimShortIdIter(t *testing.T) {
|
|||
}
|
||||
defer toDefer()
|
||||
|
||||
ch := dbpkg.ClaimShortIdIter(db, normalName, claimId)
|
||||
ch := db.ClaimShortIdIter(normalName, claimId)
|
||||
|
||||
for row := range ch {
|
||||
key := row.Key.(*prefixes.ClaimShortIDKey)
|
||||
|
@ -489,7 +489,7 @@ func TestGetTXOToClaim(t *testing.T) {
|
|||
t.Error(err)
|
||||
}
|
||||
defer toDefer()
|
||||
val, err := dbpkg.GetCachedClaimHash(db, txNum, position)
|
||||
val, err := db.GetCachedClaimHash(txNum, position)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
} else if val.Name != "one" {
|
||||
|
@ -518,7 +518,7 @@ func TestGetClaimToChannel(t *testing.T) {
|
|||
}
|
||||
defer toDefer()
|
||||
|
||||
val, err = dbpkg.GetChannelForClaim(db, claimHash, txNum, position)
|
||||
val, err = db.GetChannelForClaim(claimHash, txNum, position)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -526,7 +526,7 @@ func TestGetClaimToChannel(t *testing.T) {
|
|||
t.Errorf("Expected nil, got %s", hex.EncodeToString(val))
|
||||
}
|
||||
|
||||
val, err = dbpkg.GetChannelForClaim(db, streamHash, streamTxNum, streamPosition)
|
||||
val, err = db.GetChannelForClaim(streamHash, streamTxNum, streamPosition)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -548,7 +548,7 @@ func TestGetEffectiveAmount(t *testing.T) {
|
|||
defer toDefer()
|
||||
db.Height = 1116054
|
||||
|
||||
amount, err := dbpkg.GetEffectiveAmount(db, claimHash, true)
|
||||
amount, err := db.GetEffectiveAmount(claimHash, true)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -571,7 +571,7 @@ func TestGetSupportAmount(t *testing.T) {
|
|||
t.Error(err)
|
||||
}
|
||||
defer toDefer()
|
||||
res, err := dbpkg.GetSupportAmount(db, claimHash)
|
||||
res, err := db.GetSupportAmount(claimHash)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -591,7 +591,7 @@ func TestGetTxHash(t *testing.T) {
|
|||
t.Error(err)
|
||||
}
|
||||
defer toDefer()
|
||||
resHash, err := dbpkg.GetTxHash(db, txNum)
|
||||
resHash, err := db.GetTxHash(txNum)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -633,7 +633,7 @@ func TestGetActivation(t *testing.T) {
|
|||
t.Error(err)
|
||||
}
|
||||
defer toDefer()
|
||||
activation, err := dbpkg.GetActivation(db, txNum, position)
|
||||
activation, err := db.GetActivation(txNum, position)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -665,7 +665,7 @@ func TestGetClaimToTXO(t *testing.T) {
|
|||
return
|
||||
}
|
||||
defer toDefer()
|
||||
res, err := dbpkg.GetCachedClaimTxo(db, claimHash, true)
|
||||
res, err := db.GetCachedClaimTxo(claimHash, true)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
|
@ -694,7 +694,7 @@ func TestGetControllingClaim(t *testing.T) {
|
|||
return
|
||||
}
|
||||
defer toDefer()
|
||||
res, err := dbpkg.GetControllingClaim(db, claimName)
|
||||
res, err := db.GetControllingClaim(claimName)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
|
187
db/iterator.go
Normal file
187
db/iterator.go
Normal file
|
@ -0,0 +1,187 @@
|
|||
package db
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
|
||||
"github.com/lbryio/hub/db/prefixes"
|
||||
"github.com/linxGnu/grocksdb"
|
||||
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type IterOptions struct {
|
||||
FillCache bool
|
||||
Prefix []byte
|
||||
Start []byte //interface{}
|
||||
Stop []byte //interface{}
|
||||
IncludeStart bool
|
||||
IncludeStop bool
|
||||
IncludeKey bool
|
||||
IncludeValue bool
|
||||
RawKey bool
|
||||
RawValue bool
|
||||
CfHandle *grocksdb.ColumnFamilyHandle
|
||||
It *grocksdb.Iterator
|
||||
}
|
||||
|
||||
// NewIterateOptions creates a defualt options structure for a db iterator.
|
||||
func NewIterateOptions() *IterOptions {
|
||||
return &IterOptions{
|
||||
FillCache: false,
|
||||
Prefix: []byte{},
|
||||
Start: nil,
|
||||
Stop: nil,
|
||||
IncludeStart: true,
|
||||
IncludeStop: false,
|
||||
IncludeKey: true,
|
||||
IncludeValue: false,
|
||||
RawKey: false,
|
||||
RawValue: false,
|
||||
CfHandle: nil,
|
||||
It: nil,
|
||||
}
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithCfHandle(cfHandle *grocksdb.ColumnFamilyHandle) *IterOptions {
|
||||
o.CfHandle = cfHandle
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithFillCache(fillCache bool) *IterOptions {
|
||||
o.FillCache = fillCache
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithPrefix(prefix []byte) *IterOptions {
|
||||
o.Prefix = prefix
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithStart(start []byte) *IterOptions {
|
||||
o.Start = start
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithStop(stop []byte) *IterOptions {
|
||||
o.Stop = stop
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeStart(includeStart bool) *IterOptions {
|
||||
o.IncludeStart = includeStart
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeStop(includeStop bool) *IterOptions {
|
||||
o.IncludeStop = includeStop
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeKey(includeKey bool) *IterOptions {
|
||||
o.IncludeKey = includeKey
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithIncludeValue(includeValue bool) *IterOptions {
|
||||
o.IncludeValue = includeValue
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithRawKey(rawKey bool) *IterOptions {
|
||||
o.RawKey = rawKey
|
||||
return o
|
||||
}
|
||||
|
||||
func (o *IterOptions) WithRawValue(rawValue bool) *IterOptions {
|
||||
o.RawValue = rawValue
|
||||
return o
|
||||
}
|
||||
|
||||
func (opts *IterOptions) ReadRow(prevKey *[]byte) *prefixes.PrefixRowKV {
|
||||
it := opts.It
|
||||
if !it.Valid() {
|
||||
log.Trace("ReadRow iterator not valid returning nil")
|
||||
return nil
|
||||
}
|
||||
|
||||
key := it.Key()
|
||||
keyData := key.Data()
|
||||
keyLen := len(keyData)
|
||||
value := it.Value()
|
||||
valueData := value.Data()
|
||||
valueLen := len(valueData)
|
||||
|
||||
var outKey interface{} = nil
|
||||
var outValue interface{} = nil
|
||||
var err error = nil
|
||||
|
||||
log.Trace("keyData:", keyData)
|
||||
log.Trace("valueData:", valueData)
|
||||
|
||||
// We need to check the current key if we're not including the stop
|
||||
// key.
|
||||
if !opts.IncludeStop && opts.StopIteration(keyData) {
|
||||
log.Trace("ReadRow returning nil")
|
||||
return nil
|
||||
}
|
||||
|
||||
// We have to copy the key no matter what because we need to check
|
||||
// it on the next iterations to see if we're going to stop.
|
||||
newKeyData := make([]byte, keyLen)
|
||||
copy(newKeyData, keyData)
|
||||
if opts.IncludeKey && !opts.RawKey {
|
||||
outKey, err = prefixes.UnpackGenericKey(newKeyData)
|
||||
if err != nil {
|
||||
log.Error(err)
|
||||
}
|
||||
} else if opts.IncludeKey {
|
||||
outKey = newKeyData
|
||||
}
|
||||
|
||||
// Value could be quite large, so this setting could be important
|
||||
// for performance in some cases.
|
||||
if opts.IncludeValue {
|
||||
newValueData := make([]byte, valueLen)
|
||||
copy(newValueData, valueData)
|
||||
if !opts.RawValue {
|
||||
outValue, err = prefixes.UnpackGenericValue(newKeyData, newValueData)
|
||||
if err != nil {
|
||||
log.Error(err)
|
||||
}
|
||||
} else {
|
||||
outValue = newValueData
|
||||
}
|
||||
}
|
||||
|
||||
key.Free()
|
||||
value.Free()
|
||||
|
||||
kv := &prefixes.PrefixRowKV{
|
||||
Key: outKey,
|
||||
Value: outValue,
|
||||
}
|
||||
*prevKey = newKeyData
|
||||
|
||||
return kv
|
||||
}
|
||||
|
||||
func (o *IterOptions) StopIteration(key []byte) bool {
|
||||
if key == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
// TODO: Look at not doing floating point conversions for this
|
||||
maxLenStop := intMin(len(key), len(o.Stop))
|
||||
maxLenStart := intMin(len(key), len(o.Start))
|
||||
if o.Stop != nil &&
|
||||
(bytes.HasPrefix(key, o.Stop) || bytes.Compare(o.Stop, key[:maxLenStop]) < 0) {
|
||||
return true
|
||||
} else if o.Start != nil &&
|
||||
bytes.Compare(o.Start, key[:maxLenStart]) > 0 {
|
||||
return true
|
||||
} else if o.Prefix != nil && !bytes.HasPrefix(key, o.Prefix) {
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
|
@ -591,6 +591,13 @@ type BlockHashValue struct {
|
|||
BlockHash []byte `json:"block_hash"`
|
||||
}
|
||||
|
||||
func NewBlockHashKey(height uint32) *BlockHashKey {
|
||||
return &BlockHashKey{
|
||||
Prefix: []byte{BlockHash},
|
||||
Height: height,
|
||||
}
|
||||
}
|
||||
|
||||
func (k *BlockHashKey) PackKey() []byte {
|
||||
prefixLen := 1
|
||||
// b'>L'
|
||||
|
|
6
internal/types.go
Normal file
6
internal/types.go
Normal file
|
@ -0,0 +1,6 @@
|
|||
package internal
|
||||
|
||||
type HeightHash struct {
|
||||
Height uint64
|
||||
BlockHash []byte
|
||||
}
|
2
main.go
2
main.go
|
@ -46,7 +46,7 @@ func main() {
|
|||
|
||||
s.EsClient.Stop()
|
||||
s.GrpcServer.GracefulStop()
|
||||
db.Shutdown(s.DB)
|
||||
s.DB.Shutdown()
|
||||
|
||||
log.Println("Returning from main...")
|
||||
}()
|
||||
|
|
|
@ -15,6 +15,7 @@ service Hub {
|
|||
rpc Features(EmptyMessage) returns (StringValue) {}
|
||||
rpc Broadcast(EmptyMessage) returns (UInt32Value) {}
|
||||
rpc Height(EmptyMessage) returns (UInt32Value) {}
|
||||
rpc HeightSubscribe(UInt32Value) returns (stream UInt32Value) {}
|
||||
rpc Resolve(StringArray) returns (Outputs) {}
|
||||
}
|
||||
|
||||
|
|
|
@ -1175,7 +1175,7 @@ var file_hub_proto_rawDesc = []byte{
|
|||
0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x39, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x72, 0x65,
|
||||
0x6d, 0x6f, 0x76, 0x65, 0x44, 0x75, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x1b,
|
||||
0x0a, 0x09, 0x6e, 0x6f, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x73, 0x18, 0x3a, 0x20, 0x01, 0x28,
|
||||
0x08, 0x52, 0x08, 0x6e, 0x6f, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x73, 0x32, 0xe2, 0x03, 0x0a, 0x03,
|
||||
0x08, 0x52, 0x08, 0x6e, 0x6f, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x73, 0x32, 0x9b, 0x04, 0x0a, 0x03,
|
||||
0x48, 0x75, 0x62, 0x12, 0x2a, 0x0a, 0x06, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x12, 0x11, 0x2e,
|
||||
0x70, 0x62, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x1a, 0x0b, 0x2e, 0x70, 0x62, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, 0x00, 0x12,
|
||||
|
@ -1203,12 +1203,16 @@ var file_hub_proto_rawDesc = []byte{
|
|||
0x61, 0x6c, 0x75, 0x65, 0x22, 0x00, 0x12, 0x2d, 0x0a, 0x06, 0x48, 0x65, 0x69, 0x67, 0x68, 0x74,
|
||||
0x12, 0x10, 0x2e, 0x70, 0x62, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61,
|
||||
0x67, 0x65, 0x1a, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x33, 0x32, 0x56, 0x61,
|
||||
0x6c, 0x75, 0x65, 0x22, 0x00, 0x12, 0x29, 0x0a, 0x07, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65,
|
||||
0x12, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x41, 0x72, 0x72, 0x61,
|
||||
0x79, 0x1a, 0x0b, 0x2e, 0x70, 0x62, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, 0x00,
|
||||
0x42, 0x26, 0x5a, 0x24, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6c,
|
||||
0x62, 0x72, 0x79, 0x69, 0x6f, 0x2f, 0x68, 0x75, 0x62, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
|
||||
0x75, 0x66, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x6c, 0x75, 0x65, 0x22, 0x00, 0x12, 0x37, 0x0a, 0x0f, 0x48, 0x65, 0x69, 0x67, 0x68, 0x74, 0x53,
|
||||
0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x55, 0x49,
|
||||
0x6e, 0x74, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x55,
|
||||
0x49, 0x6e, 0x74, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x29,
|
||||
0x0a, 0x07, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x12, 0x0f, 0x2e, 0x70, 0x62, 0x2e, 0x53,
|
||||
0x74, 0x72, 0x69, 0x6e, 0x67, 0x41, 0x72, 0x72, 0x61, 0x79, 0x1a, 0x0b, 0x2e, 0x70, 0x62, 0x2e,
|
||||
0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, 0x00, 0x42, 0x26, 0x5a, 0x24, 0x67, 0x69, 0x74,
|
||||
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6c, 0x62, 0x72, 0x79, 0x69, 0x6f, 0x2f, 0x68,
|
||||
0x75, 0x62, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x67, 0x6f, 0x2f, 0x70,
|
||||
0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
@ -1272,19 +1276,21 @@ var file_hub_proto_depIdxs = []int32{
|
|||
1, // 29: pb.Hub.Features:input_type -> pb.EmptyMessage
|
||||
1, // 30: pb.Hub.Broadcast:input_type -> pb.EmptyMessage
|
||||
1, // 31: pb.Hub.Height:input_type -> pb.EmptyMessage
|
||||
6, // 32: pb.Hub.Resolve:input_type -> pb.StringArray
|
||||
11, // 33: pb.Hub.Search:output_type -> pb.Outputs
|
||||
5, // 34: pb.Hub.Ping:output_type -> pb.StringValue
|
||||
3, // 35: pb.Hub.Hello:output_type -> pb.HelloMessage
|
||||
5, // 36: pb.Hub.AddPeer:output_type -> pb.StringValue
|
||||
5, // 37: pb.Hub.PeerSubscribe:output_type -> pb.StringValue
|
||||
5, // 38: pb.Hub.Version:output_type -> pb.StringValue
|
||||
5, // 39: pb.Hub.Features:output_type -> pb.StringValue
|
||||
8, // 40: pb.Hub.Broadcast:output_type -> pb.UInt32Value
|
||||
8, // 41: pb.Hub.Height:output_type -> pb.UInt32Value
|
||||
11, // 42: pb.Hub.Resolve:output_type -> pb.Outputs
|
||||
33, // [33:43] is the sub-list for method output_type
|
||||
23, // [23:33] is the sub-list for method input_type
|
||||
8, // 32: pb.Hub.HeightSubscribe:input_type -> pb.UInt32Value
|
||||
6, // 33: pb.Hub.Resolve:input_type -> pb.StringArray
|
||||
11, // 34: pb.Hub.Search:output_type -> pb.Outputs
|
||||
5, // 35: pb.Hub.Ping:output_type -> pb.StringValue
|
||||
3, // 36: pb.Hub.Hello:output_type -> pb.HelloMessage
|
||||
5, // 37: pb.Hub.AddPeer:output_type -> pb.StringValue
|
||||
5, // 38: pb.Hub.PeerSubscribe:output_type -> pb.StringValue
|
||||
5, // 39: pb.Hub.Version:output_type -> pb.StringValue
|
||||
5, // 40: pb.Hub.Features:output_type -> pb.StringValue
|
||||
8, // 41: pb.Hub.Broadcast:output_type -> pb.UInt32Value
|
||||
8, // 42: pb.Hub.Height:output_type -> pb.UInt32Value
|
||||
8, // 43: pb.Hub.HeightSubscribe:output_type -> pb.UInt32Value
|
||||
11, // 44: pb.Hub.Resolve:output_type -> pb.Outputs
|
||||
34, // [34:45] is the sub-list for method output_type
|
||||
23, // [23:34] is the sub-list for method input_type
|
||||
23, // [23:23] is the sub-list for extension type_name
|
||||
23, // [23:23] is the sub-list for extension extendee
|
||||
0, // [0:23] is the sub-list for field type_name
|
||||
|
|
|
@ -1,4 +1,8 @@
|
|||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
||||
// versions:
|
||||
// - protoc-gen-go-grpc v1.2.0
|
||||
// - protoc v3.17.1
|
||||
// source: hub.proto
|
||||
|
||||
package pb
|
||||
|
||||
|
@ -27,6 +31,7 @@ type HubClient interface {
|
|||
Features(ctx context.Context, in *EmptyMessage, opts ...grpc.CallOption) (*StringValue, error)
|
||||
Broadcast(ctx context.Context, in *EmptyMessage, opts ...grpc.CallOption) (*UInt32Value, error)
|
||||
Height(ctx context.Context, in *EmptyMessage, opts ...grpc.CallOption) (*UInt32Value, error)
|
||||
HeightSubscribe(ctx context.Context, in *UInt32Value, opts ...grpc.CallOption) (Hub_HeightSubscribeClient, error)
|
||||
Resolve(ctx context.Context, in *StringArray, opts ...grpc.CallOption) (*Outputs, error)
|
||||
}
|
||||
|
||||
|
@ -119,6 +124,38 @@ func (c *hubClient) Height(ctx context.Context, in *EmptyMessage, opts ...grpc.C
|
|||
return out, nil
|
||||
}
|
||||
|
||||
func (c *hubClient) HeightSubscribe(ctx context.Context, in *UInt32Value, opts ...grpc.CallOption) (Hub_HeightSubscribeClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &Hub_ServiceDesc.Streams[0], "/pb.Hub/HeightSubscribe", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &hubHeightSubscribeClient{stream}
|
||||
if err := x.ClientStream.SendMsg(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := x.ClientStream.CloseSend(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type Hub_HeightSubscribeClient interface {
|
||||
Recv() (*UInt32Value, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type hubHeightSubscribeClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *hubHeightSubscribeClient) Recv() (*UInt32Value, error) {
|
||||
m := new(UInt32Value)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (c *hubClient) Resolve(ctx context.Context, in *StringArray, opts ...grpc.CallOption) (*Outputs, error) {
|
||||
out := new(Outputs)
|
||||
err := c.cc.Invoke(ctx, "/pb.Hub/Resolve", in, out, opts...)
|
||||
|
@ -141,6 +178,7 @@ type HubServer interface {
|
|||
Features(context.Context, *EmptyMessage) (*StringValue, error)
|
||||
Broadcast(context.Context, *EmptyMessage) (*UInt32Value, error)
|
||||
Height(context.Context, *EmptyMessage) (*UInt32Value, error)
|
||||
HeightSubscribe(*UInt32Value, Hub_HeightSubscribeServer) error
|
||||
Resolve(context.Context, *StringArray) (*Outputs, error)
|
||||
mustEmbedUnimplementedHubServer()
|
||||
}
|
||||
|
@ -176,6 +214,9 @@ func (UnimplementedHubServer) Broadcast(context.Context, *EmptyMessage) (*UInt32
|
|||
func (UnimplementedHubServer) Height(context.Context, *EmptyMessage) (*UInt32Value, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Height not implemented")
|
||||
}
|
||||
func (UnimplementedHubServer) HeightSubscribe(*UInt32Value, Hub_HeightSubscribeServer) error {
|
||||
return status.Errorf(codes.Unimplemented, "method HeightSubscribe not implemented")
|
||||
}
|
||||
func (UnimplementedHubServer) Resolve(context.Context, *StringArray) (*Outputs, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Resolve not implemented")
|
||||
}
|
||||
|
@ -354,6 +395,27 @@ func _Hub_Height_Handler(srv interface{}, ctx context.Context, dec func(interfac
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Hub_HeightSubscribe_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
m := new(UInt32Value)
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(HubServer).HeightSubscribe(m, &hubHeightSubscribeServer{stream})
|
||||
}
|
||||
|
||||
type Hub_HeightSubscribeServer interface {
|
||||
Send(*UInt32Value) error
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type hubHeightSubscribeServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *hubHeightSubscribeServer) Send(m *UInt32Value) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func _Hub_Resolve_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(StringArray)
|
||||
if err := dec(in); err != nil {
|
||||
|
@ -420,6 +482,12 @@ var Hub_ServiceDesc = grpc.ServiceDesc{
|
|||
Handler: _Hub_Resolve_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
Streams: []grpc.StreamDesc{
|
||||
{
|
||||
StreamName: "HeightSubscribe",
|
||||
Handler: _Hub_HeightSubscribe_Handler,
|
||||
ServerStreams: true,
|
||||
},
|
||||
},
|
||||
Metadata: "hub.proto",
|
||||
}
|
||||
|
|
173
protobuf/python/claim_pb2.py
Normal file
173
protobuf/python/claim_pb2.py
Normal file
File diff suppressed because one or more lines are too long
4
protobuf/python/claim_pb2_grpc.py
Normal file
4
protobuf/python/claim_pb2_grpc.py
Normal file
|
@ -0,0 +1,4 @@
|
|||
# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT!
|
||||
"""Client and server classes corresponding to protobuf-defined services."""
|
||||
import grpc
|
||||
|
|
@ -3,6 +3,7 @@
|
|||
# source: hub.proto
|
||||
"""Generated protocol buffer code."""
|
||||
from google.protobuf import descriptor as _descriptor
|
||||
from google.protobuf import descriptor_pool as _descriptor_pool
|
||||
from google.protobuf import message as _message
|
||||
from google.protobuf import reflection as _reflection
|
||||
from google.protobuf import symbol_database as _symbol_database
|
||||
|
@ -14,814 +15,21 @@ _sym_db = _symbol_database.Default()
|
|||
import result_pb2 as result__pb2
|
||||
|
||||
|
||||
DESCRIPTOR = _descriptor.FileDescriptor(
|
||||
name='hub.proto',
|
||||
package='pb',
|
||||
syntax='proto3',
|
||||
serialized_options=b'Z$github.com/lbryio/hub/protobuf/go/pb',
|
||||
create_key=_descriptor._internal_create_key,
|
||||
serialized_pb=b'\n\thub.proto\x12\x02pb\x1a\x0cresult.proto\"\x0e\n\x0c\x45mptyMessage\".\n\rServerMessage\x12\x0f\n\x07\x61\x64\x64ress\x18\x01 \x01(\t\x12\x0c\n\x04port\x18\x02 \x01(\t\"N\n\x0cHelloMessage\x12\x0c\n\x04port\x18\x01 \x01(\t\x12\x0c\n\x04host\x18\x02 \x01(\t\x12\"\n\x07servers\x18\x03 \x03(\x0b\x32\x11.pb.ServerMessage\"0\n\x0fInvertibleField\x12\x0e\n\x06invert\x18\x01 \x01(\x08\x12\r\n\x05value\x18\x02 \x03(\t\"\x1c\n\x0bStringValue\x12\r\n\x05value\x18\x01 \x01(\t\"\x1c\n\x0bStringArray\x12\r\n\x05value\x18\x01 \x03(\t\"\x1a\n\tBoolValue\x12\r\n\x05value\x18\x01 \x01(\x08\"\x1c\n\x0bUInt32Value\x12\r\n\x05value\x18\x01 \x01(\r\"j\n\nRangeField\x12\x1d\n\x02op\x18\x01 \x01(\x0e\x32\x11.pb.RangeField.Op\x12\r\n\x05value\x18\x02 \x03(\x05\".\n\x02Op\x12\x06\n\x02\x45Q\x10\x00\x12\x07\n\x03LTE\x10\x01\x12\x07\n\x03GTE\x10\x02\x12\x06\n\x02LT\x10\x03\x12\x06\n\x02GT\x10\x04\"\xfd\x0b\n\rSearchRequest\x12%\n\x08\x63laim_id\x18\x01 \x01(\x0b\x32\x13.pb.InvertibleField\x12\'\n\nchannel_id\x18\x02 \x01(\x0b\x32\x13.pb.InvertibleField\x12\x0c\n\x04text\x18\x03 \x01(\t\x12\r\n\x05limit\x18\x04 \x01(\x05\x12\x10\n\x08order_by\x18\x05 \x03(\t\x12\x0e\n\x06offset\x18\x06 \x01(\r\x12\x16\n\x0eis_controlling\x18\x07 \x01(\x08\x12\x1d\n\x15last_take_over_height\x18\x08 \x01(\t\x12\x12\n\nclaim_name\x18\t \x01(\t\x12\x17\n\x0fnormalized_name\x18\n \x01(\t\x12#\n\x0btx_position\x18\x0b \x03(\x0b\x32\x0e.pb.RangeField\x12\x1e\n\x06\x61mount\x18\x0c \x03(\x0b\x32\x0e.pb.RangeField\x12!\n\ttimestamp\x18\r \x03(\x0b\x32\x0e.pb.RangeField\x12*\n\x12\x63reation_timestamp\x18\x0e \x03(\x0b\x32\x0e.pb.RangeField\x12\x1e\n\x06height\x18\x0f \x03(\x0b\x32\x0e.pb.RangeField\x12\'\n\x0f\x63reation_height\x18\x10 \x03(\x0b\x32\x0e.pb.RangeField\x12)\n\x11\x61\x63tivation_height\x18\x11 \x03(\x0b\x32\x0e.pb.RangeField\x12)\n\x11\x65xpiration_height\x18\x12 \x03(\x0b\x32\x0e.pb.RangeField\x12$\n\x0crelease_time\x18\x13 \x03(\x0b\x32\x0e.pb.RangeField\x12\x11\n\tshort_url\x18\x14 \x01(\t\x12\x15\n\rcanonical_url\x18\x15 \x01(\t\x12\r\n\x05title\x18\x16 \x01(\t\x12\x0e\n\x06\x61uthor\x18\x17 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x18 \x01(\t\x12\x12\n\nclaim_type\x18\x19 \x03(\t\x12$\n\x0crepost_count\x18\x1a \x03(\x0b\x32\x0e.pb.RangeField\x12\x13\n\x0bstream_type\x18\x1b \x03(\t\x12\x12\n\nmedia_type\x18\x1c \x03(\t\x12\"\n\nfee_amount\x18\x1d \x03(\x0b\x32\x0e.pb.RangeField\x12\x14\n\x0c\x66\x65\x65_currency\x18\x1e \x01(\t\x12 \n\x08\x64uration\x18\x1f \x03(\x0b\x32\x0e.pb.RangeField\x12\x19\n\x11reposted_claim_id\x18 \x01(\t\x12#\n\x0b\x63\x65nsor_type\x18! \x03(\x0b\x32\x0e.pb.RangeField\x12\x19\n\x11\x63laims_in_channel\x18\" \x01(\t\x12)\n\x12is_signature_valid\x18$ \x01(\x0b\x32\r.pb.BoolValue\x12(\n\x10\x65\x66\x66\x65\x63tive_amount\x18% \x03(\x0b\x32\x0e.pb.RangeField\x12&\n\x0esupport_amount\x18& \x03(\x0b\x32\x0e.pb.RangeField\x12&\n\x0etrending_score\x18\' \x03(\x0b\x32\x0e.pb.RangeField\x12\r\n\x05tx_id\x18+ \x01(\t\x12 \n\x07tx_nout\x18, \x01(\x0b\x32\x0f.pb.UInt32Value\x12\x11\n\tsignature\x18- \x01(\t\x12\x18\n\x10signature_digest\x18. \x01(\t\x12\x18\n\x10public_key_bytes\x18/ \x01(\t\x12\x15\n\rpublic_key_id\x18\x30 \x01(\t\x12\x10\n\x08\x61ny_tags\x18\x31 \x03(\t\x12\x10\n\x08\x61ll_tags\x18\x32 \x03(\t\x12\x10\n\x08not_tags\x18\x33 \x03(\t\x12\x1d\n\x15has_channel_signature\x18\x34 \x01(\x08\x12!\n\nhas_source\x18\x35 \x01(\x0b\x32\r.pb.BoolValue\x12 \n\x18limit_claims_per_channel\x18\x36 \x01(\x05\x12\x15\n\rany_languages\x18\x37 \x03(\t\x12\x15\n\rall_languages\x18\x38 \x03(\t\x12\x19\n\x11remove_duplicates\x18\x39 \x01(\x08\x12\x11\n\tno_totals\x18: \x01(\x08\x32\xe2\x03\n\x03Hub\x12*\n\x06Search\x12\x11.pb.SearchRequest\x1a\x0b.pb.Outputs\"\x00\x12+\n\x04Ping\x12\x10.pb.EmptyMessage\x1a\x0f.pb.StringValue\"\x00\x12-\n\x05Hello\x12\x10.pb.HelloMessage\x1a\x10.pb.HelloMessage\"\x00\x12/\n\x07\x41\x64\x64Peer\x12\x11.pb.ServerMessage\x1a\x0f.pb.StringValue\"\x00\x12\x35\n\rPeerSubscribe\x12\x11.pb.ServerMessage\x1a\x0f.pb.StringValue\"\x00\x12.\n\x07Version\x12\x10.pb.EmptyMessage\x1a\x0f.pb.StringValue\"\x00\x12/\n\x08\x46\x65\x61tures\x12\x10.pb.EmptyMessage\x1a\x0f.pb.StringValue\"\x00\x12\x30\n\tBroadcast\x12\x10.pb.EmptyMessage\x1a\x0f.pb.UInt32Value\"\x00\x12-\n\x06Height\x12\x10.pb.EmptyMessage\x1a\x0f.pb.UInt32Value\"\x00\x12)\n\x07Resolve\x12\x0f.pb.StringArray\x1a\x0b.pb.Outputs\"\x00\x42&Z$github.com/lbryio/hub/protobuf/go/pbb\x06proto3'
|
||||
,
|
||||
dependencies=[result__pb2.DESCRIPTOR,])
|
||||
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\thub.proto\x12\x02pb\x1a\x0cresult.proto\"\x0e\n\x0c\x45mptyMessage\".\n\rServerMessage\x12\x0f\n\x07\x61\x64\x64ress\x18\x01 \x01(\t\x12\x0c\n\x04port\x18\x02 \x01(\t\"N\n\x0cHelloMessage\x12\x0c\n\x04port\x18\x01 \x01(\t\x12\x0c\n\x04host\x18\x02 \x01(\t\x12\"\n\x07servers\x18\x03 \x03(\x0b\x32\x11.pb.ServerMessage\"0\n\x0fInvertibleField\x12\x0e\n\x06invert\x18\x01 \x01(\x08\x12\r\n\x05value\x18\x02 \x03(\t\"\x1c\n\x0bStringValue\x12\r\n\x05value\x18\x01 \x01(\t\"\x1c\n\x0bStringArray\x12\r\n\x05value\x18\x01 \x03(\t\"\x1a\n\tBoolValue\x12\r\n\x05value\x18\x01 \x01(\x08\"\x1c\n\x0bUInt32Value\x12\r\n\x05value\x18\x01 \x01(\r\"j\n\nRangeField\x12\x1d\n\x02op\x18\x01 \x01(\x0e\x32\x11.pb.RangeField.Op\x12\r\n\x05value\x18\x02 \x03(\x05\".\n\x02Op\x12\x06\n\x02\x45Q\x10\x00\x12\x07\n\x03LTE\x10\x01\x12\x07\n\x03GTE\x10\x02\x12\x06\n\x02LT\x10\x03\x12\x06\n\x02GT\x10\x04\"\xfd\x0b\n\rSearchRequest\x12%\n\x08\x63laim_id\x18\x01 \x01(\x0b\x32\x13.pb.InvertibleField\x12\'\n\nchannel_id\x18\x02 \x01(\x0b\x32\x13.pb.InvertibleField\x12\x0c\n\x04text\x18\x03 \x01(\t\x12\r\n\x05limit\x18\x04 \x01(\x05\x12\x10\n\x08order_by\x18\x05 \x03(\t\x12\x0e\n\x06offset\x18\x06 \x01(\r\x12\x16\n\x0eis_controlling\x18\x07 \x01(\x08\x12\x1d\n\x15last_take_over_height\x18\x08 \x01(\t\x12\x12\n\nclaim_name\x18\t \x01(\t\x12\x17\n\x0fnormalized_name\x18\n \x01(\t\x12#\n\x0btx_position\x18\x0b \x03(\x0b\x32\x0e.pb.RangeField\x12\x1e\n\x06\x61mount\x18\x0c \x03(\x0b\x32\x0e.pb.RangeField\x12!\n\ttimestamp\x18\r \x03(\x0b\x32\x0e.pb.RangeField\x12*\n\x12\x63reation_timestamp\x18\x0e \x03(\x0b\x32\x0e.pb.RangeField\x12\x1e\n\x06height\x18\x0f \x03(\x0b\x32\x0e.pb.RangeField\x12\'\n\x0f\x63reation_height\x18\x10 \x03(\x0b\x32\x0e.pb.RangeField\x12)\n\x11\x61\x63tivation_height\x18\x11 \x03(\x0b\x32\x0e.pb.RangeField\x12)\n\x11\x65xpiration_height\x18\x12 \x03(\x0b\x32\x0e.pb.RangeField\x12$\n\x0crelease_time\x18\x13 \x03(\x0b\x32\x0e.pb.RangeField\x12\x11\n\tshort_url\x18\x14 \x01(\t\x12\x15\n\rcanonical_url\x18\x15 \x01(\t\x12\r\n\x05title\x18\x16 \x01(\t\x12\x0e\n\x06\x61uthor\x18\x17 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x18 \x01(\t\x12\x12\n\nclaim_type\x18\x19 \x03(\t\x12$\n\x0crepost_count\x18\x1a \x03(\x0b\x32\x0e.pb.RangeField\x12\x13\n\x0bstream_type\x18\x1b \x03(\t\x12\x12\n\nmedia_type\x18\x1c \x03(\t\x12\"\n\nfee_amount\x18\x1d \x03(\x0b\x32\x0e.pb.RangeField\x12\x14\n\x0c\x66\x65\x65_currency\x18\x1e \x01(\t\x12 \n\x08\x64uration\x18\x1f \x03(\x0b\x32\x0e.pb.RangeField\x12\x19\n\x11reposted_claim_id\x18 \x01(\t\x12#\n\x0b\x63\x65nsor_type\x18! \x03(\x0b\x32\x0e.pb.RangeField\x12\x19\n\x11\x63laims_in_channel\x18\" \x01(\t\x12)\n\x12is_signature_valid\x18$ \x01(\x0b\x32\r.pb.BoolValue\x12(\n\x10\x65\x66\x66\x65\x63tive_amount\x18% \x03(\x0b\x32\x0e.pb.RangeField\x12&\n\x0esupport_amount\x18& \x03(\x0b\x32\x0e.pb.RangeField\x12&\n\x0etrending_score\x18\' \x03(\x0b\x32\x0e.pb.RangeField\x12\r\n\x05tx_id\x18+ \x01(\t\x12 \n\x07tx_nout\x18, \x01(\x0b\x32\x0f.pb.UInt32Value\x12\x11\n\tsignature\x18- \x01(\t\x12\x18\n\x10signature_digest\x18. \x01(\t\x12\x18\n\x10public_key_bytes\x18/ \x01(\t\x12\x15\n\rpublic_key_id\x18\x30 \x01(\t\x12\x10\n\x08\x61ny_tags\x18\x31 \x03(\t\x12\x10\n\x08\x61ll_tags\x18\x32 \x03(\t\x12\x10\n\x08not_tags\x18\x33 \x03(\t\x12\x1d\n\x15has_channel_signature\x18\x34 \x01(\x08\x12!\n\nhas_source\x18\x35 \x01(\x0b\x32\r.pb.BoolValue\x12 \n\x18limit_claims_per_channel\x18\x36 \x01(\x05\x12\x15\n\rany_languages\x18\x37 \x03(\t\x12\x15\n\rall_languages\x18\x38 \x03(\t\x12\x19\n\x11remove_duplicates\x18\x39 \x01(\x08\x12\x11\n\tno_totals\x18: \x01(\x08\x32\x9b\x04\n\x03Hub\x12*\n\x06Search\x12\x11.pb.SearchRequest\x1a\x0b.pb.Outputs\"\x00\x12+\n\x04Ping\x12\x10.pb.EmptyMessage\x1a\x0f.pb.StringValue\"\x00\x12-\n\x05Hello\x12\x10.pb.HelloMessage\x1a\x10.pb.HelloMessage\"\x00\x12/\n\x07\x41\x64\x64Peer\x12\x11.pb.ServerMessage\x1a\x0f.pb.StringValue\"\x00\x12\x35\n\rPeerSubscribe\x12\x11.pb.ServerMessage\x1a\x0f.pb.StringValue\"\x00\x12.\n\x07Version\x12\x10.pb.EmptyMessage\x1a\x0f.pb.StringValue\"\x00\x12/\n\x08\x46\x65\x61tures\x12\x10.pb.EmptyMessage\x1a\x0f.pb.StringValue\"\x00\x12\x30\n\tBroadcast\x12\x10.pb.EmptyMessage\x1a\x0f.pb.UInt32Value\"\x00\x12-\n\x06Height\x12\x10.pb.EmptyMessage\x1a\x0f.pb.UInt32Value\"\x00\x12\x37\n\x0fHeightSubscribe\x12\x0f.pb.UInt32Value\x1a\x0f.pb.UInt32Value\"\x00\x30\x01\x12)\n\x07Resolve\x12\x0f.pb.StringArray\x1a\x0b.pb.Outputs\"\x00\x42&Z$github.com/lbryio/hub/protobuf/go/pbb\x06proto3')
|
||||
|
||||
|
||||
|
||||
_RANGEFIELD_OP = _descriptor.EnumDescriptor(
|
||||
name='Op',
|
||||
full_name='pb.RangeField.Op',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
values=[
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='EQ', index=0, number=0,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='LTE', index=1, number=1,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='GTE', index=2, number=2,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='LT', index=3, number=3,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='GT', index=4, number=4,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
containing_type=None,
|
||||
serialized_options=None,
|
||||
serialized_start=403,
|
||||
serialized_end=449,
|
||||
)
|
||||
_sym_db.RegisterEnumDescriptor(_RANGEFIELD_OP)
|
||||
|
||||
|
||||
_EMPTYMESSAGE = _descriptor.Descriptor(
|
||||
name='EmptyMessage',
|
||||
full_name='pb.EmptyMessage',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=31,
|
||||
serialized_end=45,
|
||||
)
|
||||
|
||||
|
||||
_SERVERMESSAGE = _descriptor.Descriptor(
|
||||
name='ServerMessage',
|
||||
full_name='pb.ServerMessage',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='address', full_name='pb.ServerMessage.address', index=0,
|
||||
number=1, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='port', full_name='pb.ServerMessage.port', index=1,
|
||||
number=2, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=47,
|
||||
serialized_end=93,
|
||||
)
|
||||
|
||||
|
||||
_HELLOMESSAGE = _descriptor.Descriptor(
|
||||
name='HelloMessage',
|
||||
full_name='pb.HelloMessage',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='port', full_name='pb.HelloMessage.port', index=0,
|
||||
number=1, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='host', full_name='pb.HelloMessage.host', index=1,
|
||||
number=2, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='servers', full_name='pb.HelloMessage.servers', index=2,
|
||||
number=3, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=95,
|
||||
serialized_end=173,
|
||||
)
|
||||
|
||||
|
||||
_INVERTIBLEFIELD = _descriptor.Descriptor(
|
||||
name='InvertibleField',
|
||||
full_name='pb.InvertibleField',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='invert', full_name='pb.InvertibleField.invert', index=0,
|
||||
number=1, type=8, cpp_type=7, label=1,
|
||||
has_default_value=False, default_value=False,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='value', full_name='pb.InvertibleField.value', index=1,
|
||||
number=2, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=175,
|
||||
serialized_end=223,
|
||||
)
|
||||
|
||||
|
||||
_STRINGVALUE = _descriptor.Descriptor(
|
||||
name='StringValue',
|
||||
full_name='pb.StringValue',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='value', full_name='pb.StringValue.value', index=0,
|
||||
number=1, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=225,
|
||||
serialized_end=253,
|
||||
)
|
||||
|
||||
|
||||
_STRINGARRAY = _descriptor.Descriptor(
|
||||
name='StringArray',
|
||||
full_name='pb.StringArray',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='value', full_name='pb.StringArray.value', index=0,
|
||||
number=1, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=255,
|
||||
serialized_end=283,
|
||||
)
|
||||
|
||||
|
||||
_BOOLVALUE = _descriptor.Descriptor(
|
||||
name='BoolValue',
|
||||
full_name='pb.BoolValue',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='value', full_name='pb.BoolValue.value', index=0,
|
||||
number=1, type=8, cpp_type=7, label=1,
|
||||
has_default_value=False, default_value=False,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=285,
|
||||
serialized_end=311,
|
||||
)
|
||||
|
||||
|
||||
_UINT32VALUE = _descriptor.Descriptor(
|
||||
name='UInt32Value',
|
||||
full_name='pb.UInt32Value',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='value', full_name='pb.UInt32Value.value', index=0,
|
||||
number=1, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=313,
|
||||
serialized_end=341,
|
||||
)
|
||||
|
||||
|
||||
_RANGEFIELD = _descriptor.Descriptor(
|
||||
name='RangeField',
|
||||
full_name='pb.RangeField',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='op', full_name='pb.RangeField.op', index=0,
|
||||
number=1, type=14, cpp_type=8, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='value', full_name='pb.RangeField.value', index=1,
|
||||
number=2, type=5, cpp_type=1, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
_RANGEFIELD_OP,
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=343,
|
||||
serialized_end=449,
|
||||
)
|
||||
|
||||
|
||||
_SEARCHREQUEST = _descriptor.Descriptor(
|
||||
name='SearchRequest',
|
||||
full_name='pb.SearchRequest',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='claim_id', full_name='pb.SearchRequest.claim_id', index=0,
|
||||
number=1, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='channel_id', full_name='pb.SearchRequest.channel_id', index=1,
|
||||
number=2, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='text', full_name='pb.SearchRequest.text', index=2,
|
||||
number=3, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='limit', full_name='pb.SearchRequest.limit', index=3,
|
||||
number=4, type=5, cpp_type=1, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='order_by', full_name='pb.SearchRequest.order_by', index=4,
|
||||
number=5, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='offset', full_name='pb.SearchRequest.offset', index=5,
|
||||
number=6, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='is_controlling', full_name='pb.SearchRequest.is_controlling', index=6,
|
||||
number=7, type=8, cpp_type=7, label=1,
|
||||
has_default_value=False, default_value=False,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='last_take_over_height', full_name='pb.SearchRequest.last_take_over_height', index=7,
|
||||
number=8, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='claim_name', full_name='pb.SearchRequest.claim_name', index=8,
|
||||
number=9, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='normalized_name', full_name='pb.SearchRequest.normalized_name', index=9,
|
||||
number=10, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='tx_position', full_name='pb.SearchRequest.tx_position', index=10,
|
||||
number=11, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='amount', full_name='pb.SearchRequest.amount', index=11,
|
||||
number=12, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='timestamp', full_name='pb.SearchRequest.timestamp', index=12,
|
||||
number=13, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='creation_timestamp', full_name='pb.SearchRequest.creation_timestamp', index=13,
|
||||
number=14, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='height', full_name='pb.SearchRequest.height', index=14,
|
||||
number=15, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='creation_height', full_name='pb.SearchRequest.creation_height', index=15,
|
||||
number=16, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='activation_height', full_name='pb.SearchRequest.activation_height', index=16,
|
||||
number=17, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='expiration_height', full_name='pb.SearchRequest.expiration_height', index=17,
|
||||
number=18, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='release_time', full_name='pb.SearchRequest.release_time', index=18,
|
||||
number=19, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='short_url', full_name='pb.SearchRequest.short_url', index=19,
|
||||
number=20, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='canonical_url', full_name='pb.SearchRequest.canonical_url', index=20,
|
||||
number=21, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='title', full_name='pb.SearchRequest.title', index=21,
|
||||
number=22, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='author', full_name='pb.SearchRequest.author', index=22,
|
||||
number=23, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='description', full_name='pb.SearchRequest.description', index=23,
|
||||
number=24, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='claim_type', full_name='pb.SearchRequest.claim_type', index=24,
|
||||
number=25, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='repost_count', full_name='pb.SearchRequest.repost_count', index=25,
|
||||
number=26, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='stream_type', full_name='pb.SearchRequest.stream_type', index=26,
|
||||
number=27, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='media_type', full_name='pb.SearchRequest.media_type', index=27,
|
||||
number=28, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='fee_amount', full_name='pb.SearchRequest.fee_amount', index=28,
|
||||
number=29, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='fee_currency', full_name='pb.SearchRequest.fee_currency', index=29,
|
||||
number=30, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='duration', full_name='pb.SearchRequest.duration', index=30,
|
||||
number=31, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='reposted_claim_id', full_name='pb.SearchRequest.reposted_claim_id', index=31,
|
||||
number=32, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='censor_type', full_name='pb.SearchRequest.censor_type', index=32,
|
||||
number=33, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='claims_in_channel', full_name='pb.SearchRequest.claims_in_channel', index=33,
|
||||
number=34, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='is_signature_valid', full_name='pb.SearchRequest.is_signature_valid', index=34,
|
||||
number=36, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='effective_amount', full_name='pb.SearchRequest.effective_amount', index=35,
|
||||
number=37, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='support_amount', full_name='pb.SearchRequest.support_amount', index=36,
|
||||
number=38, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='trending_score', full_name='pb.SearchRequest.trending_score', index=37,
|
||||
number=39, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='tx_id', full_name='pb.SearchRequest.tx_id', index=38,
|
||||
number=43, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='tx_nout', full_name='pb.SearchRequest.tx_nout', index=39,
|
||||
number=44, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='signature', full_name='pb.SearchRequest.signature', index=40,
|
||||
number=45, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='signature_digest', full_name='pb.SearchRequest.signature_digest', index=41,
|
||||
number=46, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='public_key_bytes', full_name='pb.SearchRequest.public_key_bytes', index=42,
|
||||
number=47, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='public_key_id', full_name='pb.SearchRequest.public_key_id', index=43,
|
||||
number=48, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='any_tags', full_name='pb.SearchRequest.any_tags', index=44,
|
||||
number=49, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='all_tags', full_name='pb.SearchRequest.all_tags', index=45,
|
||||
number=50, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='not_tags', full_name='pb.SearchRequest.not_tags', index=46,
|
||||
number=51, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='has_channel_signature', full_name='pb.SearchRequest.has_channel_signature', index=47,
|
||||
number=52, type=8, cpp_type=7, label=1,
|
||||
has_default_value=False, default_value=False,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='has_source', full_name='pb.SearchRequest.has_source', index=48,
|
||||
number=53, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='limit_claims_per_channel', full_name='pb.SearchRequest.limit_claims_per_channel', index=49,
|
||||
number=54, type=5, cpp_type=1, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='any_languages', full_name='pb.SearchRequest.any_languages', index=50,
|
||||
number=55, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='all_languages', full_name='pb.SearchRequest.all_languages', index=51,
|
||||
number=56, type=9, cpp_type=9, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='remove_duplicates', full_name='pb.SearchRequest.remove_duplicates', index=52,
|
||||
number=57, type=8, cpp_type=7, label=1,
|
||||
has_default_value=False, default_value=False,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='no_totals', full_name='pb.SearchRequest.no_totals', index=53,
|
||||
number=58, type=8, cpp_type=7, label=1,
|
||||
has_default_value=False, default_value=False,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=452,
|
||||
serialized_end=1985,
|
||||
)
|
||||
|
||||
_HELLOMESSAGE.fields_by_name['servers'].message_type = _SERVERMESSAGE
|
||||
_RANGEFIELD.fields_by_name['op'].enum_type = _RANGEFIELD_OP
|
||||
_RANGEFIELD_OP.containing_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['claim_id'].message_type = _INVERTIBLEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['channel_id'].message_type = _INVERTIBLEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['tx_position'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['amount'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['timestamp'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['creation_timestamp'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['height'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['creation_height'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['activation_height'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['expiration_height'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['release_time'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['repost_count'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['fee_amount'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['duration'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['censor_type'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['is_signature_valid'].message_type = _BOOLVALUE
|
||||
_SEARCHREQUEST.fields_by_name['effective_amount'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['support_amount'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['trending_score'].message_type = _RANGEFIELD
|
||||
_SEARCHREQUEST.fields_by_name['tx_nout'].message_type = _UINT32VALUE
|
||||
_SEARCHREQUEST.fields_by_name['has_source'].message_type = _BOOLVALUE
|
||||
DESCRIPTOR.message_types_by_name['EmptyMessage'] = _EMPTYMESSAGE
|
||||
DESCRIPTOR.message_types_by_name['ServerMessage'] = _SERVERMESSAGE
|
||||
DESCRIPTOR.message_types_by_name['HelloMessage'] = _HELLOMESSAGE
|
||||
DESCRIPTOR.message_types_by_name['InvertibleField'] = _INVERTIBLEFIELD
|
||||
DESCRIPTOR.message_types_by_name['StringValue'] = _STRINGVALUE
|
||||
DESCRIPTOR.message_types_by_name['StringArray'] = _STRINGARRAY
|
||||
DESCRIPTOR.message_types_by_name['BoolValue'] = _BOOLVALUE
|
||||
DESCRIPTOR.message_types_by_name['UInt32Value'] = _UINT32VALUE
|
||||
DESCRIPTOR.message_types_by_name['RangeField'] = _RANGEFIELD
|
||||
DESCRIPTOR.message_types_by_name['SearchRequest'] = _SEARCHREQUEST
|
||||
_sym_db.RegisterFileDescriptor(DESCRIPTOR)
|
||||
|
||||
_EMPTYMESSAGE = DESCRIPTOR.message_types_by_name['EmptyMessage']
|
||||
_SERVERMESSAGE = DESCRIPTOR.message_types_by_name['ServerMessage']
|
||||
_HELLOMESSAGE = DESCRIPTOR.message_types_by_name['HelloMessage']
|
||||
_INVERTIBLEFIELD = DESCRIPTOR.message_types_by_name['InvertibleField']
|
||||
_STRINGVALUE = DESCRIPTOR.message_types_by_name['StringValue']
|
||||
_STRINGARRAY = DESCRIPTOR.message_types_by_name['StringArray']
|
||||
_BOOLVALUE = DESCRIPTOR.message_types_by_name['BoolValue']
|
||||
_UINT32VALUE = DESCRIPTOR.message_types_by_name['UInt32Value']
|
||||
_RANGEFIELD = DESCRIPTOR.message_types_by_name['RangeField']
|
||||
_SEARCHREQUEST = DESCRIPTOR.message_types_by_name['SearchRequest']
|
||||
_RANGEFIELD_OP = _RANGEFIELD.enum_types_by_name['Op']
|
||||
EmptyMessage = _reflection.GeneratedProtocolMessageType('EmptyMessage', (_message.Message,), {
|
||||
'DESCRIPTOR' : _EMPTYMESSAGE,
|
||||
'__module__' : 'hub_pb2'
|
||||
|
@ -892,122 +100,33 @@ SearchRequest = _reflection.GeneratedProtocolMessageType('SearchRequest', (_mess
|
|||
})
|
||||
_sym_db.RegisterMessage(SearchRequest)
|
||||
|
||||
_HUB = DESCRIPTOR.services_by_name['Hub']
|
||||
if _descriptor._USE_C_DESCRIPTORS == False:
|
||||
|
||||
DESCRIPTOR._options = None
|
||||
|
||||
_HUB = _descriptor.ServiceDescriptor(
|
||||
name='Hub',
|
||||
full_name='pb.Hub',
|
||||
file=DESCRIPTOR,
|
||||
index=0,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
serialized_start=1988,
|
||||
serialized_end=2470,
|
||||
methods=[
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Search',
|
||||
full_name='pb.Hub.Search',
|
||||
index=0,
|
||||
containing_service=None,
|
||||
input_type=_SEARCHREQUEST,
|
||||
output_type=result__pb2._OUTPUTS,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Ping',
|
||||
full_name='pb.Hub.Ping',
|
||||
index=1,
|
||||
containing_service=None,
|
||||
input_type=_EMPTYMESSAGE,
|
||||
output_type=_STRINGVALUE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Hello',
|
||||
full_name='pb.Hub.Hello',
|
||||
index=2,
|
||||
containing_service=None,
|
||||
input_type=_HELLOMESSAGE,
|
||||
output_type=_HELLOMESSAGE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='AddPeer',
|
||||
full_name='pb.Hub.AddPeer',
|
||||
index=3,
|
||||
containing_service=None,
|
||||
input_type=_SERVERMESSAGE,
|
||||
output_type=_STRINGVALUE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='PeerSubscribe',
|
||||
full_name='pb.Hub.PeerSubscribe',
|
||||
index=4,
|
||||
containing_service=None,
|
||||
input_type=_SERVERMESSAGE,
|
||||
output_type=_STRINGVALUE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Version',
|
||||
full_name='pb.Hub.Version',
|
||||
index=5,
|
||||
containing_service=None,
|
||||
input_type=_EMPTYMESSAGE,
|
||||
output_type=_STRINGVALUE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Features',
|
||||
full_name='pb.Hub.Features',
|
||||
index=6,
|
||||
containing_service=None,
|
||||
input_type=_EMPTYMESSAGE,
|
||||
output_type=_STRINGVALUE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Broadcast',
|
||||
full_name='pb.Hub.Broadcast',
|
||||
index=7,
|
||||
containing_service=None,
|
||||
input_type=_EMPTYMESSAGE,
|
||||
output_type=_UINT32VALUE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Height',
|
||||
full_name='pb.Hub.Height',
|
||||
index=8,
|
||||
containing_service=None,
|
||||
input_type=_EMPTYMESSAGE,
|
||||
output_type=_UINT32VALUE,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
_descriptor.MethodDescriptor(
|
||||
name='Resolve',
|
||||
full_name='pb.Hub.Resolve',
|
||||
index=9,
|
||||
containing_service=None,
|
||||
input_type=_STRINGARRAY,
|
||||
output_type=result__pb2._OUTPUTS,
|
||||
serialized_options=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
),
|
||||
])
|
||||
_sym_db.RegisterServiceDescriptor(_HUB)
|
||||
|
||||
DESCRIPTOR.services_by_name['Hub'] = _HUB
|
||||
|
||||
DESCRIPTOR._options = None
|
||||
DESCRIPTOR._serialized_options = b'Z$github.com/lbryio/hub/protobuf/go/pb'
|
||||
_EMPTYMESSAGE._serialized_start=31
|
||||
_EMPTYMESSAGE._serialized_end=45
|
||||
_SERVERMESSAGE._serialized_start=47
|
||||
_SERVERMESSAGE._serialized_end=93
|
||||
_HELLOMESSAGE._serialized_start=95
|
||||
_HELLOMESSAGE._serialized_end=173
|
||||
_INVERTIBLEFIELD._serialized_start=175
|
||||
_INVERTIBLEFIELD._serialized_end=223
|
||||
_STRINGVALUE._serialized_start=225
|
||||
_STRINGVALUE._serialized_end=253
|
||||
_STRINGARRAY._serialized_start=255
|
||||
_STRINGARRAY._serialized_end=283
|
||||
_BOOLVALUE._serialized_start=285
|
||||
_BOOLVALUE._serialized_end=311
|
||||
_UINT32VALUE._serialized_start=313
|
||||
_UINT32VALUE._serialized_end=341
|
||||
_RANGEFIELD._serialized_start=343
|
||||
_RANGEFIELD._serialized_end=449
|
||||
_RANGEFIELD_OP._serialized_start=403
|
||||
_RANGEFIELD_OP._serialized_end=449
|
||||
_SEARCHREQUEST._serialized_start=452
|
||||
_SEARCHREQUEST._serialized_end=1985
|
||||
_HUB._serialized_start=1988
|
||||
_HUB._serialized_end=2527
|
||||
# @@protoc_insertion_point(module_scope)
|
||||
|
|
|
@ -60,6 +60,11 @@ class HubStub(object):
|
|||
request_serializer=hub__pb2.EmptyMessage.SerializeToString,
|
||||
response_deserializer=hub__pb2.UInt32Value.FromString,
|
||||
)
|
||||
self.HeightSubscribe = channel.unary_stream(
|
||||
'/pb.Hub/HeightSubscribe',
|
||||
request_serializer=hub__pb2.UInt32Value.SerializeToString,
|
||||
response_deserializer=hub__pb2.UInt32Value.FromString,
|
||||
)
|
||||
self.Resolve = channel.unary_unary(
|
||||
'/pb.Hub/Resolve',
|
||||
request_serializer=hub__pb2.StringArray.SerializeToString,
|
||||
|
@ -124,6 +129,12 @@ class HubServicer(object):
|
|||
context.set_details('Method not implemented!')
|
||||
raise NotImplementedError('Method not implemented!')
|
||||
|
||||
def HeightSubscribe(self, request, context):
|
||||
"""Missing associated documentation comment in .proto file."""
|
||||
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
|
||||
context.set_details('Method not implemented!')
|
||||
raise NotImplementedError('Method not implemented!')
|
||||
|
||||
def Resolve(self, request, context):
|
||||
"""Missing associated documentation comment in .proto file."""
|
||||
context.set_code(grpc.StatusCode.UNIMPLEMENTED)
|
||||
|
@ -178,6 +189,11 @@ def add_HubServicer_to_server(servicer, server):
|
|||
request_deserializer=hub__pb2.EmptyMessage.FromString,
|
||||
response_serializer=hub__pb2.UInt32Value.SerializeToString,
|
||||
),
|
||||
'HeightSubscribe': grpc.unary_stream_rpc_method_handler(
|
||||
servicer.HeightSubscribe,
|
||||
request_deserializer=hub__pb2.UInt32Value.FromString,
|
||||
response_serializer=hub__pb2.UInt32Value.SerializeToString,
|
||||
),
|
||||
'Resolve': grpc.unary_unary_rpc_method_handler(
|
||||
servicer.Resolve,
|
||||
request_deserializer=hub__pb2.StringArray.FromString,
|
||||
|
@ -346,6 +362,23 @@ class Hub(object):
|
|||
options, channel_credentials,
|
||||
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
|
||||
|
||||
@staticmethod
|
||||
def HeightSubscribe(request,
|
||||
target,
|
||||
options=(),
|
||||
channel_credentials=None,
|
||||
call_credentials=None,
|
||||
insecure=False,
|
||||
compression=None,
|
||||
wait_for_ready=None,
|
||||
timeout=None,
|
||||
metadata=None):
|
||||
return grpc.experimental.unary_stream(request, target, '/pb.Hub/HeightSubscribe',
|
||||
hub__pb2.UInt32Value.SerializeToString,
|
||||
hub__pb2.UInt32Value.FromString,
|
||||
options, channel_credentials,
|
||||
insecure, call_credentials, compression, wait_for_ready, timeout, metadata)
|
||||
|
||||
@staticmethod
|
||||
def Resolve(request,
|
||||
target,
|
||||
|
|
|
@ -3,6 +3,7 @@
|
|||
# source: result.proto
|
||||
"""Generated protocol buffer code."""
|
||||
from google.protobuf import descriptor as _descriptor
|
||||
from google.protobuf import descriptor_pool as _descriptor_pool
|
||||
from google.protobuf import message as _message
|
||||
from google.protobuf import reflection as _reflection
|
||||
from google.protobuf import symbol_database as _symbol_database
|
||||
|
@ -13,417 +14,16 @@ _sym_db = _symbol_database.Default()
|
|||
|
||||
|
||||
|
||||
DESCRIPTOR = _descriptor.FileDescriptor(
|
||||
name='result.proto',
|
||||
package='pb',
|
||||
syntax='proto3',
|
||||
serialized_options=b'Z$github.com/lbryio/hub/protobuf/go/pb',
|
||||
create_key=_descriptor._internal_create_key,
|
||||
serialized_pb=b'\n\x0cresult.proto\x12\x02pb\"\x97\x01\n\x07Outputs\x12\x18\n\x04txos\x18\x01 \x03(\x0b\x32\n.pb.Output\x12\x1e\n\nextra_txos\x18\x02 \x03(\x0b\x32\n.pb.Output\x12\r\n\x05total\x18\x03 \x01(\r\x12\x0e\n\x06offset\x18\x04 \x01(\r\x12\x1c\n\x07\x62locked\x18\x05 \x03(\x0b\x32\x0b.pb.Blocked\x12\x15\n\rblocked_total\x18\x06 \x01(\r\"{\n\x06Output\x12\x0f\n\x07tx_hash\x18\x01 \x01(\x0c\x12\x0c\n\x04nout\x18\x02 \x01(\r\x12\x0e\n\x06height\x18\x03 \x01(\r\x12\x1e\n\x05\x63laim\x18\x07 \x01(\x0b\x32\r.pb.ClaimMetaH\x00\x12\x1a\n\x05\x65rror\x18\x0f \x01(\x0b\x32\t.pb.ErrorH\x00\x42\x06\n\x04meta\"\xe6\x02\n\tClaimMeta\x12\x1b\n\x07\x63hannel\x18\x01 \x01(\x0b\x32\n.pb.Output\x12\x1a\n\x06repost\x18\x02 \x01(\x0b\x32\n.pb.Output\x12\x11\n\tshort_url\x18\x03 \x01(\t\x12\x15\n\rcanonical_url\x18\x04 \x01(\t\x12\x16\n\x0eis_controlling\x18\x05 \x01(\x08\x12\x18\n\x10take_over_height\x18\x06 \x01(\r\x12\x17\n\x0f\x63reation_height\x18\x07 \x01(\r\x12\x19\n\x11\x61\x63tivation_height\x18\x08 \x01(\r\x12\x19\n\x11\x65xpiration_height\x18\t \x01(\r\x12\x19\n\x11\x63laims_in_channel\x18\n \x01(\r\x12\x10\n\x08reposted\x18\x0b \x01(\r\x12\x18\n\x10\x65\x66\x66\x65\x63tive_amount\x18\x14 \x01(\x04\x12\x16\n\x0esupport_amount\x18\x15 \x01(\x04\x12\x16\n\x0etrending_score\x18\x16 \x01(\x01\"\x94\x01\n\x05\x45rror\x12\x1c\n\x04\x63ode\x18\x01 \x01(\x0e\x32\x0e.pb.Error.Code\x12\x0c\n\x04text\x18\x02 \x01(\t\x12\x1c\n\x07\x62locked\x18\x03 \x01(\x0b\x32\x0b.pb.Blocked\"A\n\x04\x43ode\x12\x10\n\x0cUNKNOWN_CODE\x10\x00\x12\r\n\tNOT_FOUND\x10\x01\x12\x0b\n\x07INVALID\x10\x02\x12\x0b\n\x07\x42LOCKED\x10\x03\"5\n\x07\x42locked\x12\r\n\x05\x63ount\x18\x01 \x01(\r\x12\x1b\n\x07\x63hannel\x18\x02 \x01(\x0b\x32\n.pb.OutputB&Z$github.com/lbryio/hub/protobuf/go/pbb\x06proto3'
|
||||
)
|
||||
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0cresult.proto\x12\x02pb\"\x97\x01\n\x07Outputs\x12\x18\n\x04txos\x18\x01 \x03(\x0b\x32\n.pb.Output\x12\x1e\n\nextra_txos\x18\x02 \x03(\x0b\x32\n.pb.Output\x12\r\n\x05total\x18\x03 \x01(\r\x12\x0e\n\x06offset\x18\x04 \x01(\r\x12\x1c\n\x07\x62locked\x18\x05 \x03(\x0b\x32\x0b.pb.Blocked\x12\x15\n\rblocked_total\x18\x06 \x01(\r\"{\n\x06Output\x12\x0f\n\x07tx_hash\x18\x01 \x01(\x0c\x12\x0c\n\x04nout\x18\x02 \x01(\r\x12\x0e\n\x06height\x18\x03 \x01(\r\x12\x1e\n\x05\x63laim\x18\x07 \x01(\x0b\x32\r.pb.ClaimMetaH\x00\x12\x1a\n\x05\x65rror\x18\x0f \x01(\x0b\x32\t.pb.ErrorH\x00\x42\x06\n\x04meta\"\xe6\x02\n\tClaimMeta\x12\x1b\n\x07\x63hannel\x18\x01 \x01(\x0b\x32\n.pb.Output\x12\x1a\n\x06repost\x18\x02 \x01(\x0b\x32\n.pb.Output\x12\x11\n\tshort_url\x18\x03 \x01(\t\x12\x15\n\rcanonical_url\x18\x04 \x01(\t\x12\x16\n\x0eis_controlling\x18\x05 \x01(\x08\x12\x18\n\x10take_over_height\x18\x06 \x01(\r\x12\x17\n\x0f\x63reation_height\x18\x07 \x01(\r\x12\x19\n\x11\x61\x63tivation_height\x18\x08 \x01(\r\x12\x19\n\x11\x65xpiration_height\x18\t \x01(\r\x12\x19\n\x11\x63laims_in_channel\x18\n \x01(\r\x12\x10\n\x08reposted\x18\x0b \x01(\r\x12\x18\n\x10\x65\x66\x66\x65\x63tive_amount\x18\x14 \x01(\x04\x12\x16\n\x0esupport_amount\x18\x15 \x01(\x04\x12\x16\n\x0etrending_score\x18\x16 \x01(\x01\"\x94\x01\n\x05\x45rror\x12\x1c\n\x04\x63ode\x18\x01 \x01(\x0e\x32\x0e.pb.Error.Code\x12\x0c\n\x04text\x18\x02 \x01(\t\x12\x1c\n\x07\x62locked\x18\x03 \x01(\x0b\x32\x0b.pb.Blocked\"A\n\x04\x43ode\x12\x10\n\x0cUNKNOWN_CODE\x10\x00\x12\r\n\tNOT_FOUND\x10\x01\x12\x0b\n\x07INVALID\x10\x02\x12\x0b\n\x07\x42LOCKED\x10\x03\"5\n\x07\x42locked\x12\r\n\x05\x63ount\x18\x01 \x01(\r\x12\x1b\n\x07\x63hannel\x18\x02 \x01(\x0b\x32\n.pb.OutputB&Z$github.com/lbryio/hub/protobuf/go/pbb\x06proto3')
|
||||
|
||||
|
||||
|
||||
_ERROR_CODE = _descriptor.EnumDescriptor(
|
||||
name='Code',
|
||||
full_name='pb.Error.Code',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
values=[
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='UNKNOWN_CODE', index=0, number=0,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='NOT_FOUND', index=1, number=1,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='INVALID', index=2, number=2,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='BLOCKED', index=3, number=3,
|
||||
serialized_options=None,
|
||||
type=None,
|
||||
create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
containing_type=None,
|
||||
serialized_options=None,
|
||||
serialized_start=744,
|
||||
serialized_end=809,
|
||||
)
|
||||
_sym_db.RegisterEnumDescriptor(_ERROR_CODE)
|
||||
|
||||
|
||||
_OUTPUTS = _descriptor.Descriptor(
|
||||
name='Outputs',
|
||||
full_name='pb.Outputs',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='txos', full_name='pb.Outputs.txos', index=0,
|
||||
number=1, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='extra_txos', full_name='pb.Outputs.extra_txos', index=1,
|
||||
number=2, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='total', full_name='pb.Outputs.total', index=2,
|
||||
number=3, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='offset', full_name='pb.Outputs.offset', index=3,
|
||||
number=4, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='blocked', full_name='pb.Outputs.blocked', index=4,
|
||||
number=5, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='blocked_total', full_name='pb.Outputs.blocked_total', index=5,
|
||||
number=6, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=21,
|
||||
serialized_end=172,
|
||||
)
|
||||
|
||||
|
||||
_OUTPUT = _descriptor.Descriptor(
|
||||
name='Output',
|
||||
full_name='pb.Output',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='tx_hash', full_name='pb.Output.tx_hash', index=0,
|
||||
number=1, type=12, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"",
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='nout', full_name='pb.Output.nout', index=1,
|
||||
number=2, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='height', full_name='pb.Output.height', index=2,
|
||||
number=3, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='claim', full_name='pb.Output.claim', index=3,
|
||||
number=7, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='error', full_name='pb.Output.error', index=4,
|
||||
number=15, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
_descriptor.OneofDescriptor(
|
||||
name='meta', full_name='pb.Output.meta',
|
||||
index=0, containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[]),
|
||||
],
|
||||
serialized_start=174,
|
||||
serialized_end=297,
|
||||
)
|
||||
|
||||
|
||||
_CLAIMMETA = _descriptor.Descriptor(
|
||||
name='ClaimMeta',
|
||||
full_name='pb.ClaimMeta',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='channel', full_name='pb.ClaimMeta.channel', index=0,
|
||||
number=1, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='repost', full_name='pb.ClaimMeta.repost', index=1,
|
||||
number=2, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='short_url', full_name='pb.ClaimMeta.short_url', index=2,
|
||||
number=3, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='canonical_url', full_name='pb.ClaimMeta.canonical_url', index=3,
|
||||
number=4, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='is_controlling', full_name='pb.ClaimMeta.is_controlling', index=4,
|
||||
number=5, type=8, cpp_type=7, label=1,
|
||||
has_default_value=False, default_value=False,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='take_over_height', full_name='pb.ClaimMeta.take_over_height', index=5,
|
||||
number=6, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='creation_height', full_name='pb.ClaimMeta.creation_height', index=6,
|
||||
number=7, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='activation_height', full_name='pb.ClaimMeta.activation_height', index=7,
|
||||
number=8, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='expiration_height', full_name='pb.ClaimMeta.expiration_height', index=8,
|
||||
number=9, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='claims_in_channel', full_name='pb.ClaimMeta.claims_in_channel', index=9,
|
||||
number=10, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='reposted', full_name='pb.ClaimMeta.reposted', index=10,
|
||||
number=11, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='effective_amount', full_name='pb.ClaimMeta.effective_amount', index=11,
|
||||
number=20, type=4, cpp_type=4, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='support_amount', full_name='pb.ClaimMeta.support_amount', index=12,
|
||||
number=21, type=4, cpp_type=4, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='trending_score', full_name='pb.ClaimMeta.trending_score', index=13,
|
||||
number=22, type=1, cpp_type=5, label=1,
|
||||
has_default_value=False, default_value=float(0),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=300,
|
||||
serialized_end=658,
|
||||
)
|
||||
|
||||
|
||||
_ERROR = _descriptor.Descriptor(
|
||||
name='Error',
|
||||
full_name='pb.Error',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='code', full_name='pb.Error.code', index=0,
|
||||
number=1, type=14, cpp_type=8, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='text', full_name='pb.Error.text', index=1,
|
||||
number=2, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=b"".decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='blocked', full_name='pb.Error.blocked', index=2,
|
||||
number=3, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
_ERROR_CODE,
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=661,
|
||||
serialized_end=809,
|
||||
)
|
||||
|
||||
|
||||
_BLOCKED = _descriptor.Descriptor(
|
||||
name='Blocked',
|
||||
full_name='pb.Blocked',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
create_key=_descriptor._internal_create_key,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='count', full_name='pb.Blocked.count', index=0,
|
||||
number=1, type=13, cpp_type=3, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='channel', full_name='pb.Blocked.channel', index=1,
|
||||
number=2, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
serialized_options=None, file=DESCRIPTOR, create_key=_descriptor._internal_create_key),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
serialized_options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=811,
|
||||
serialized_end=864,
|
||||
)
|
||||
|
||||
_OUTPUTS.fields_by_name['txos'].message_type = _OUTPUT
|
||||
_OUTPUTS.fields_by_name['extra_txos'].message_type = _OUTPUT
|
||||
_OUTPUTS.fields_by_name['blocked'].message_type = _BLOCKED
|
||||
_OUTPUT.fields_by_name['claim'].message_type = _CLAIMMETA
|
||||
_OUTPUT.fields_by_name['error'].message_type = _ERROR
|
||||
_OUTPUT.oneofs_by_name['meta'].fields.append(
|
||||
_OUTPUT.fields_by_name['claim'])
|
||||
_OUTPUT.fields_by_name['claim'].containing_oneof = _OUTPUT.oneofs_by_name['meta']
|
||||
_OUTPUT.oneofs_by_name['meta'].fields.append(
|
||||
_OUTPUT.fields_by_name['error'])
|
||||
_OUTPUT.fields_by_name['error'].containing_oneof = _OUTPUT.oneofs_by_name['meta']
|
||||
_CLAIMMETA.fields_by_name['channel'].message_type = _OUTPUT
|
||||
_CLAIMMETA.fields_by_name['repost'].message_type = _OUTPUT
|
||||
_ERROR.fields_by_name['code'].enum_type = _ERROR_CODE
|
||||
_ERROR.fields_by_name['blocked'].message_type = _BLOCKED
|
||||
_ERROR_CODE.containing_type = _ERROR
|
||||
_BLOCKED.fields_by_name['channel'].message_type = _OUTPUT
|
||||
DESCRIPTOR.message_types_by_name['Outputs'] = _OUTPUTS
|
||||
DESCRIPTOR.message_types_by_name['Output'] = _OUTPUT
|
||||
DESCRIPTOR.message_types_by_name['ClaimMeta'] = _CLAIMMETA
|
||||
DESCRIPTOR.message_types_by_name['Error'] = _ERROR
|
||||
DESCRIPTOR.message_types_by_name['Blocked'] = _BLOCKED
|
||||
_sym_db.RegisterFileDescriptor(DESCRIPTOR)
|
||||
|
||||
_OUTPUTS = DESCRIPTOR.message_types_by_name['Outputs']
|
||||
_OUTPUT = DESCRIPTOR.message_types_by_name['Output']
|
||||
_CLAIMMETA = DESCRIPTOR.message_types_by_name['ClaimMeta']
|
||||
_ERROR = DESCRIPTOR.message_types_by_name['Error']
|
||||
_BLOCKED = DESCRIPTOR.message_types_by_name['Blocked']
|
||||
_ERROR_CODE = _ERROR.enum_types_by_name['Code']
|
||||
Outputs = _reflection.GeneratedProtocolMessageType('Outputs', (_message.Message,), {
|
||||
'DESCRIPTOR' : _OUTPUTS,
|
||||
'__module__' : 'result_pb2'
|
||||
|
@ -459,6 +59,20 @@ Blocked = _reflection.GeneratedProtocolMessageType('Blocked', (_message.Message,
|
|||
})
|
||||
_sym_db.RegisterMessage(Blocked)
|
||||
|
||||
if _descriptor._USE_C_DESCRIPTORS == False:
|
||||
|
||||
DESCRIPTOR._options = None
|
||||
DESCRIPTOR._options = None
|
||||
DESCRIPTOR._serialized_options = b'Z$github.com/lbryio/hub/protobuf/go/pb'
|
||||
_OUTPUTS._serialized_start=21
|
||||
_OUTPUTS._serialized_end=172
|
||||
_OUTPUT._serialized_start=174
|
||||
_OUTPUT._serialized_end=297
|
||||
_CLAIMMETA._serialized_start=300
|
||||
_CLAIMMETA._serialized_end=658
|
||||
_ERROR._serialized_start=661
|
||||
_ERROR._serialized_end=809
|
||||
_ERROR_CODE._serialized_start=744
|
||||
_ERROR_CODE._serialized_end=809
|
||||
_BLOCKED._serialized_start=811
|
||||
_BLOCKED._serialized_end=864
|
||||
# @@protoc_insertion_point(module_scope)
|
||||
|
|
|
@ -26,13 +26,16 @@ type Args struct {
|
|||
EsHost string
|
||||
EsPort string
|
||||
PrometheusPort string
|
||||
NotifierPort string
|
||||
EsIndex string
|
||||
RefreshDelta int
|
||||
CacheTTL int
|
||||
PeerFile string
|
||||
Country string
|
||||
DisableEs bool
|
||||
BlockingChannelIds []string
|
||||
FilteringChannelIds []string
|
||||
Debug bool
|
||||
DisableEs bool
|
||||
DisableLoadPeers bool
|
||||
DisableStartPrometheus bool
|
||||
DisableStartUDP bool
|
||||
|
@ -41,8 +44,7 @@ type Args struct {
|
|||
DisableRocksDBRefresh bool
|
||||
DisableResolve bool
|
||||
DisableBlockingAndFiltering bool
|
||||
BlockingChannelIds []string
|
||||
FilteringChannelIds []string
|
||||
DisableStartNotifier bool
|
||||
}
|
||||
|
||||
const (
|
||||
|
@ -53,6 +55,7 @@ const (
|
|||
DefaultEsIndex = "claims"
|
||||
DefaultEsPort = "9200"
|
||||
DefaultPrometheusPort = "2112"
|
||||
DefaultNotifierPort = "18080"
|
||||
DefaultRefreshDelta = 5
|
||||
DefaultCacheTTL = 5
|
||||
DefaultPeerFile = "peers.txt"
|
||||
|
@ -65,6 +68,7 @@ const (
|
|||
DefaultDisableRockDBRefresh = false
|
||||
DefaultDisableResolve = false
|
||||
DefaultDisableBlockingAndFiltering = false
|
||||
DisableStartNotifier = false
|
||||
)
|
||||
|
||||
var (
|
||||
|
@ -111,6 +115,7 @@ func ParseArgs(searchRequest *pb.SearchRequest) *Args {
|
|||
esHost := parser.String("", "eshost", &argparse.Options{Required: false, Help: "elasticsearch host", Default: DefaultEsHost})
|
||||
esPort := parser.String("", "esport", &argparse.Options{Required: false, Help: "elasticsearch port", Default: DefaultEsPort})
|
||||
prometheusPort := parser.String("", "prometheus-port", &argparse.Options{Required: false, Help: "prometheus port", Default: DefaultPrometheusPort})
|
||||
notifierPort := parser.String("", "notifier-port", &argparse.Options{Required: false, Help: "notifier port", Default: DefaultNotifierPort})
|
||||
esIndex := parser.String("", "esindex", &argparse.Options{Required: false, Help: "elasticsearch index name", Default: DefaultEsIndex})
|
||||
refreshDelta := parser.Int("", "refresh-delta", &argparse.Options{Required: false, Help: "elasticsearch index refresh delta in seconds", Default: DefaultRefreshDelta})
|
||||
cacheTTL := parser.Int("", "cachettl", &argparse.Options{Required: false, Help: "Cache TTL in minutes", Default: DefaultCacheTTL})
|
||||
|
@ -129,6 +134,7 @@ func ParseArgs(searchRequest *pb.SearchRequest) *Args {
|
|||
disableRocksDBRefresh := parser.Flag("", "disable-rocksdb-refresh", &argparse.Options{Required: false, Help: "Disable rocksdb refreshing", Default: DefaultDisableRockDBRefresh})
|
||||
disableResolve := parser.Flag("", "disable-resolve", &argparse.Options{Required: false, Help: "Disable resolve endpoint (and rocksdb loading)", Default: DefaultDisableRockDBRefresh})
|
||||
disableBlockingAndFiltering := parser.Flag("", "disable-blocking-and-filtering", &argparse.Options{Required: false, Help: "Disable blocking and filtering of channels and streams", Default: DefaultDisableBlockingAndFiltering})
|
||||
disableStartNotifier := parser.Flag("", "disable-start-notifier", &argparse.Options{Required: false, Help: "Disable start notifier", Default: DisableStartNotifier})
|
||||
|
||||
text := parser.String("", "text", &argparse.Options{Required: false, Help: "text query"})
|
||||
name := parser.String("", "name", &argparse.Options{Required: false, Help: "name"})
|
||||
|
@ -154,13 +160,16 @@ func ParseArgs(searchRequest *pb.SearchRequest) *Args {
|
|||
EsHost: *esHost,
|
||||
EsPort: *esPort,
|
||||
PrometheusPort: *prometheusPort,
|
||||
NotifierPort: *notifierPort,
|
||||
EsIndex: *esIndex,
|
||||
RefreshDelta: *refreshDelta,
|
||||
CacheTTL: *cacheTTL,
|
||||
PeerFile: *peerFile,
|
||||
Country: *country,
|
||||
DisableEs: *disableEs,
|
||||
BlockingChannelIds: *blockingChannelIds,
|
||||
FilteringChannelIds: *filteringChannelIds,
|
||||
Debug: *debug,
|
||||
DisableEs: *disableEs,
|
||||
DisableLoadPeers: *disableLoadPeers,
|
||||
DisableStartPrometheus: *disableStartPrometheus,
|
||||
DisableStartUDP: *disableStartUdp,
|
||||
|
@ -169,8 +178,7 @@ func ParseArgs(searchRequest *pb.SearchRequest) *Args {
|
|||
DisableRocksDBRefresh: *disableRocksDBRefresh,
|
||||
DisableResolve: *disableResolve,
|
||||
DisableBlockingAndFiltering: *disableBlockingAndFiltering,
|
||||
BlockingChannelIds: *blockingChannelIds,
|
||||
FilteringChannelIds: *filteringChannelIds,
|
||||
DisableStartNotifier: *disableStartNotifier,
|
||||
}
|
||||
|
||||
if esHost, ok := environment["ELASTIC_HOST"]; ok {
|
||||
|
|
|
@ -44,6 +44,7 @@ func removeFile(fileName string) {
|
|||
}
|
||||
}
|
||||
|
||||
// makeDefaultArgs creates a default set of arguments for testing the server.
|
||||
func makeDefaultArgs() *server.Args {
|
||||
args := &server.Args{
|
||||
CmdType: server.ServeCmd,
|
||||
|
@ -53,6 +54,7 @@ func makeDefaultArgs() *server.Args {
|
|||
EsHost: server.DefaultEsHost,
|
||||
EsPort: server.DefaultEsPort,
|
||||
PrometheusPort: server.DefaultPrometheusPort,
|
||||
NotifierPort: server.DefaultNotifierPort,
|
||||
EsIndex: server.DefaultEsIndex,
|
||||
RefreshDelta: server.DefaultRefreshDelta,
|
||||
CacheTTL: server.DefaultCacheTTL,
|
||||
|
@ -67,6 +69,7 @@ func makeDefaultArgs() *server.Args {
|
|||
DisableRocksDBRefresh: true,
|
||||
DisableResolve: true,
|
||||
DisableBlockingAndFiltering: true,
|
||||
DisableStartNotifier: true,
|
||||
}
|
||||
|
||||
return args
|
||||
|
|
97
server/notifier.go
Normal file
97
server/notifier.go
Normal file
|
@ -0,0 +1,97 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"net"
|
||||
|
||||
"github.com/lbryio/hub/internal"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
const NotifierResponseLength = 40
|
||||
|
||||
// AddHeightSub adds a new height subscriber
|
||||
func (s *Server) AddHeightSub(addr net.Addr, conn net.Conn) {
|
||||
s.HeightSubsMut.Lock()
|
||||
defer s.HeightSubsMut.Unlock()
|
||||
s.HeightSubs[addr] = conn
|
||||
}
|
||||
|
||||
// DoNotify sends a notification to all height subscribers
|
||||
func (s *Server) DoNotify(heightHash *internal.HeightHash) error {
|
||||
buff := make([]byte, NotifierResponseLength)
|
||||
toDelete := make([]net.Addr, 0)
|
||||
|
||||
s.HeightSubsMut.RLock()
|
||||
for addr, conn := range s.HeightSubs {
|
||||
// struct.pack(b'>Q32s', height, block_hash)
|
||||
binary.BigEndian.PutUint64(buff, heightHash.Height)
|
||||
copy(buff[8:], heightHash.BlockHash[:32])
|
||||
logrus.Tracef("notifying %s", addr)
|
||||
n, err := conn.Write(buff)
|
||||
if err != nil {
|
||||
logrus.Warn(err)
|
||||
toDelete = append(toDelete, addr)
|
||||
}
|
||||
if n != NotifierResponseLength {
|
||||
logrus.Warn("not all bytes written")
|
||||
}
|
||||
}
|
||||
s.HeightSubsMut.RUnlock()
|
||||
|
||||
if len(toDelete) > 0 {
|
||||
s.HeightSubsMut.Lock()
|
||||
for _, v := range toDelete {
|
||||
delete(s.HeightSubs, v)
|
||||
}
|
||||
s.HeightSubsMut.Unlock()
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// RunNotifier Runs the notfying action forever
|
||||
func (s *Server) RunNotifier() error {
|
||||
for heightHash := range s.NotifierChan {
|
||||
s.DoNotify(heightHash)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// NotifierServer implements the TCP protocol for height/blockheader notifications
|
||||
func (s *Server) NotifierServer() error {
|
||||
address := ":" + s.Args.NotifierPort
|
||||
addr, err := net.ResolveTCPAddr("tcp", address)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
listen, err := net.ListenTCP("tcp", addr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
defer listen.Close()
|
||||
|
||||
for {
|
||||
|
||||
logrus.Info("Waiting for connection")
|
||||
conn, err := listen.Accept()
|
||||
if err != nil {
|
||||
logrus.Warn(err)
|
||||
continue
|
||||
}
|
||||
|
||||
addr := conn.RemoteAddr()
|
||||
|
||||
logrus.Println(addr)
|
||||
|
||||
// _, err = conn.Write([]byte(addr.String()))
|
||||
// if err != nil {
|
||||
// logrus.Warn(err)
|
||||
// continue
|
||||
// }
|
||||
|
||||
go s.AddHeightSub(addr, conn)
|
||||
}
|
||||
}
|
87
server/notifier_test.go
Normal file
87
server/notifier_test.go
Normal file
|
@ -0,0 +1,87 @@
|
|||
package server_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"net"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/lbryio/hub/internal"
|
||||
"github.com/lbryio/hub/server"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
const defaultBufferSize = 1024
|
||||
|
||||
func tcpConnReady(addr string) (net.Conn, error) {
|
||||
sleepTime := time.Millisecond * 100
|
||||
for {
|
||||
if sleepTime > time.Second {
|
||||
return nil, fmt.Errorf("timeout")
|
||||
}
|
||||
|
||||
conn, err := net.Dial("tcp", addr)
|
||||
if err != nil {
|
||||
logrus.Warn(err)
|
||||
time.Sleep(sleepTime)
|
||||
sleepTime = sleepTime * 2
|
||||
continue
|
||||
}
|
||||
return conn, nil
|
||||
}
|
||||
}
|
||||
|
||||
func tcpRead(conn net.Conn) ([]byte, error) {
|
||||
buf := make([]byte, defaultBufferSize)
|
||||
n, err := conn.Read(buf)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if n != server.NotifierResponseLength {
|
||||
return nil, fmt.Errorf("not all bytes read")
|
||||
}
|
||||
|
||||
return buf[:n], nil
|
||||
}
|
||||
|
||||
func TestNotifierServer(t *testing.T) {
|
||||
args := makeDefaultArgs()
|
||||
ctx := context.Background()
|
||||
hub := server.MakeHubServer(ctx, args)
|
||||
|
||||
go hub.NotifierServer()
|
||||
go hub.RunNotifier()
|
||||
|
||||
addr := fmt.Sprintf(":%s", args.NotifierPort)
|
||||
logrus.Info(addr)
|
||||
conn, err := tcpConnReady(addr)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
resCh := make(chan []byte)
|
||||
|
||||
go func() {
|
||||
logrus.Warn("waiting for response")
|
||||
res, err := tcpRead(conn)
|
||||
logrus.Warn("got response")
|
||||
if err != nil {
|
||||
logrus.Warn(err)
|
||||
return
|
||||
}
|
||||
resCh <- res
|
||||
}()
|
||||
|
||||
// Hacky but needed because if the reader isn't ready
|
||||
// before the writer sends it won't get the data
|
||||
time.Sleep(time.Second)
|
||||
|
||||
hash, _ := hex.DecodeString("AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA")
|
||||
logrus.Warn("sending hash")
|
||||
hub.NotifierChan <- &internal.HeightHash{Height: 1, BlockHash: hash}
|
||||
|
||||
res := <-resCh
|
||||
logrus.Info(string(res))
|
||||
}
|
219
server/server.go
219
server/server.go
|
@ -6,6 +6,7 @@ import (
|
|||
"encoding/hex"
|
||||
"fmt"
|
||||
"hash"
|
||||
"io/ioutil"
|
||||
"log"
|
||||
"net"
|
||||
"net/http"
|
||||
|
@ -16,6 +17,7 @@ import (
|
|||
|
||||
"github.com/ReneKroon/ttlcache/v2"
|
||||
"github.com/lbryio/hub/db"
|
||||
"github.com/lbryio/hub/internal"
|
||||
"github.com/lbryio/hub/internal/metrics"
|
||||
"github.com/lbryio/hub/meta"
|
||||
pb "github.com/lbryio/hub/protobuf/go"
|
||||
|
@ -46,6 +48,9 @@ type Server struct {
|
|||
PeerSubsMut sync.RWMutex
|
||||
NumPeerSubs *int64
|
||||
ExternalIP net.IP
|
||||
HeightSubs map[net.Addr]net.Conn
|
||||
HeightSubsMut sync.RWMutex
|
||||
NotifierChan chan *internal.HeightHash
|
||||
pb.UnimplementedHubServer
|
||||
}
|
||||
|
||||
|
@ -92,6 +97,7 @@ func getVersion() string {
|
|||
'blockchain.address.unsubscribe'
|
||||
*/
|
||||
|
||||
// PeerSubsLoadOrStore thread safe load or store for peer subs
|
||||
func (s *Server) PeerSubsLoadOrStore(peer *Peer) (actual *Peer, loaded bool) {
|
||||
key := peer.peerKey()
|
||||
s.PeerSubsMut.RLock()
|
||||
|
@ -107,6 +113,7 @@ func (s *Server) PeerSubsLoadOrStore(peer *Peer) (actual *Peer, loaded bool) {
|
|||
}
|
||||
}
|
||||
|
||||
// PeerServersLoadOrStore thread safe load or store for peer servers
|
||||
func (s *Server) PeerServersLoadOrStore(peer *Peer) (actual *Peer, loaded bool) {
|
||||
key := peer.peerKey()
|
||||
s.PeerServersMut.RLock()
|
||||
|
@ -122,6 +129,7 @@ func (s *Server) PeerServersLoadOrStore(peer *Peer) (actual *Peer, loaded bool)
|
|||
}
|
||||
}
|
||||
|
||||
// Run "main" function for starting the server. This blocks.
|
||||
func (s *Server) Run() {
|
||||
l, err := net.Listen("tcp", ":"+s.Args.Port)
|
||||
if err != nil {
|
||||
|
@ -192,14 +200,22 @@ func MakeHubServer(ctx context.Context, args *Args) *Server {
|
|||
var myDB *db.ReadOnlyDBColumnFamily
|
||||
// var dbShutdown = func() {}
|
||||
if !args.DisableResolve {
|
||||
tmpName := fmt.Sprintf("/tmp/%d", time.Now().Nanosecond())
|
||||
logrus.Warn("tmpName", tmpName)
|
||||
tmpName, err := ioutil.TempDir("", "go-lbry-hub")
|
||||
if err != nil {
|
||||
logrus.Info(err)
|
||||
log.Fatal(err)
|
||||
}
|
||||
logrus.Info("tmpName", tmpName)
|
||||
if err != nil {
|
||||
logrus.Info(err)
|
||||
}
|
||||
myDB, _, err = db.GetProdDB(args.DBPath, tmpName)
|
||||
// dbShutdown = func() {
|
||||
// db.Shutdown(myDB)
|
||||
// }
|
||||
if err != nil {
|
||||
// Can't load the db, fail loudly
|
||||
logrus.Info(err)
|
||||
log.Fatalln(err)
|
||||
}
|
||||
|
||||
|
@ -245,26 +261,44 @@ func MakeHubServer(ctx context.Context, args *Args) *Server {
|
|||
PeerSubsMut: sync.RWMutex{},
|
||||
NumPeerSubs: numSubs,
|
||||
ExternalIP: net.IPv4(127, 0, 0, 1),
|
||||
HeightSubs: make(map[net.Addr]net.Conn),
|
||||
HeightSubsMut: sync.RWMutex{},
|
||||
NotifierChan: make(chan *internal.HeightHash),
|
||||
}
|
||||
|
||||
// Start up our background services
|
||||
if !args.DisableResolve && !args.DisableRocksDBRefresh {
|
||||
db.RunDetectChanges(myDB)
|
||||
logrus.Info("Running detect changes")
|
||||
myDB.RunDetectChanges(s.NotifierChan)
|
||||
}
|
||||
if !args.DisableBlockingAndFiltering {
|
||||
db.RunGetBlocksAndFiltes(myDB)
|
||||
myDB.RunGetBlocksAndFilters()
|
||||
}
|
||||
if !args.DisableStartPrometheus {
|
||||
go s.prometheusEndpoint(s.Args.PrometheusPort, "metrics")
|
||||
}
|
||||
if !args.DisableStartUDP {
|
||||
go func() {
|
||||
err := UDPServer(args)
|
||||
err := s.UDPServer()
|
||||
if err != nil {
|
||||
log.Println("UDP Server failed!", err)
|
||||
}
|
||||
}()
|
||||
}
|
||||
if !args.DisableStartNotifier {
|
||||
go func() {
|
||||
err := s.NotifierServer()
|
||||
if err != nil {
|
||||
log.Println("Notifier Server failed!", err)
|
||||
}
|
||||
}()
|
||||
go func() {
|
||||
err := s.RunNotifier()
|
||||
if err != nil {
|
||||
log.Println("RunNotifier failed!", err)
|
||||
}
|
||||
}()
|
||||
}
|
||||
// Load peers from disk and subscribe to one if there are any
|
||||
if !args.DisableLoadPeers {
|
||||
go func() {
|
||||
|
@ -372,146 +406,36 @@ func (s *Server) Height(ctx context.Context, args *pb.EmptyMessage) (*pb.UInt32V
|
|||
}
|
||||
}
|
||||
|
||||
func ResolveResultToOutput(res *db.ResolveResult) *pb.Output {
|
||||
// func ResolveResultToOutput(res *db.ResolveResult, outputType byte) *OutputWType {
|
||||
// res.ClaimHash
|
||||
var channelOutput *pb.Output
|
||||
var repostOutput *pb.Output
|
||||
|
||||
if res.ChannelTxHash != nil {
|
||||
channelOutput = &pb.Output{
|
||||
TxHash: res.ChannelTxHash,
|
||||
Nout: uint32(res.ChannelTxPostition),
|
||||
Height: res.ChannelHeight,
|
||||
// HeightSubscribe takes a height to wait for the server to reach and waits until it reaches that
|
||||
// height or higher and returns the current height. If the db is off it will return 0.
|
||||
func (s *Server) HeightSubscribe(arg *pb.UInt32Value, stream pb.Hub_HeightSubscribeServer) error {
|
||||
metrics.RequestsCount.With(prometheus.Labels{"method": "height"}).Inc()
|
||||
if s.DB != nil {
|
||||
want := arg.Value
|
||||
for s.DB.LastState.Height < want {
|
||||
if s.DB.LastState.Height >= want {
|
||||
err := stream.Send(&pb.UInt32Value{Value: s.DB.LastState.Height})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
time.Sleep(time.Millisecond * 100)
|
||||
}
|
||||
} else {
|
||||
if err := stream.Send(&pb.UInt32Value{Value: 0}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
if res.RepostTxHash != nil {
|
||||
repostOutput = &pb.Output{
|
||||
TxHash: res.RepostTxHash,
|
||||
Nout: uint32(res.RepostTxPostition),
|
||||
Height: res.RepostHeight,
|
||||
}
|
||||
}
|
||||
|
||||
claimMeta := &pb.ClaimMeta{
|
||||
Channel: channelOutput,
|
||||
Repost: repostOutput,
|
||||
ShortUrl: res.ShortUrl,
|
||||
Reposted: uint32(res.Reposted),
|
||||
IsControlling: res.IsControlling,
|
||||
CreationHeight: res.CreationHeight,
|
||||
ExpirationHeight: res.ExpirationHeight,
|
||||
ClaimsInChannel: res.ClaimsInChannel,
|
||||
EffectiveAmount: res.EffectiveAmount,
|
||||
SupportAmount: res.SupportAmount,
|
||||
}
|
||||
|
||||
claim := &pb.Output_Claim{
|
||||
Claim: claimMeta,
|
||||
}
|
||||
|
||||
output := &pb.Output{
|
||||
TxHash: res.TxHash,
|
||||
Nout: uint32(res.Position),
|
||||
Height: res.Height,
|
||||
Meta: claim,
|
||||
}
|
||||
|
||||
// outputWType := &OutputWType{
|
||||
// Output: output,
|
||||
// OutputType: outputType,
|
||||
// }
|
||||
|
||||
return output
|
||||
return nil
|
||||
}
|
||||
|
||||
func ExpandedResolveResultToOutput(res *db.ExpandedResolveResult) ([]*pb.Output, []*pb.Output, error) {
|
||||
// func ExpandedResolveResultToOutput(res *db.ExpandedResolveResult) ([]*OutputWType, []*OutputWType, error) {
|
||||
// FIXME: Set references in extraTxos properly
|
||||
// FIXME: figure out the handling of rows and extra properly
|
||||
// FIXME: want to return empty list or nil when extraTxos is empty?
|
||||
txos := make([]*pb.Output, 0)
|
||||
extraTxos := make([]*pb.Output, 0)
|
||||
// txos := make([]*OutputWType, 0)
|
||||
// extraTxos := make([]*OutputWType, 0)
|
||||
// Errors
|
||||
if x := res.Channel.GetError(); x != nil {
|
||||
logrus.Warn("Channel error: ", x)
|
||||
outputErr := &pb.Output_Error{
|
||||
Error: &pb.Error{
|
||||
Text: x.Error.Error(),
|
||||
Code: pb.Error_Code(x.ErrorType), //FIXME
|
||||
},
|
||||
}
|
||||
// res := &OutputWType{
|
||||
// Output: &pb.Output{Meta: outputErr},
|
||||
// OutputType: OutputErrorType,
|
||||
// }
|
||||
res := &pb.Output{Meta: outputErr}
|
||||
txos = append(txos, res)
|
||||
return txos, nil, nil
|
||||
}
|
||||
if x := res.Stream.GetError(); x != nil {
|
||||
logrus.Warn("Stream error: ", x)
|
||||
outputErr := &pb.Output_Error{
|
||||
Error: &pb.Error{
|
||||
Text: x.Error.Error(),
|
||||
Code: pb.Error_Code(x.ErrorType), //FIXME
|
||||
},
|
||||
}
|
||||
// res := &OutputWType{
|
||||
// Output: &pb.Output{Meta: outputErr},
|
||||
// OutputType: OutputErrorType,
|
||||
// }
|
||||
res := &pb.Output{Meta: outputErr}
|
||||
txos = append(txos, res)
|
||||
return txos, nil, nil
|
||||
}
|
||||
// HeightHashSubscribe takes a height to wait for the server to reach and waits until it reaches that
|
||||
// height or higher and returns the current height. If the db is off it will return 0.
|
||||
func (s *Server) HeightHashSubscribe() error {
|
||||
metrics.RequestsCount.With(prometheus.Labels{"method": "height_hash"}).Inc()
|
||||
|
||||
// Not errors
|
||||
var channel, stream, repost, repostedChannel *db.ResolveResult
|
||||
|
||||
channel = res.Channel.GetResult()
|
||||
stream = res.Stream.GetResult()
|
||||
repost = res.Repost.GetResult()
|
||||
repostedChannel = res.RepostedChannel.GetResult()
|
||||
|
||||
if channel != nil && stream == nil {
|
||||
// Channel
|
||||
output := ResolveResultToOutput(channel)
|
||||
txos = append(txos, output)
|
||||
|
||||
if repost != nil {
|
||||
output := ResolveResultToOutput(repost)
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
if repostedChannel != nil {
|
||||
output := ResolveResultToOutput(repostedChannel)
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
|
||||
return txos, extraTxos, nil
|
||||
} else if stream != nil {
|
||||
output := ResolveResultToOutput(stream)
|
||||
txos = append(txos, output)
|
||||
if channel != nil {
|
||||
output := ResolveResultToOutput(channel)
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
if repost != nil {
|
||||
output := ResolveResultToOutput(repost)
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
if repostedChannel != nil {
|
||||
output := ResolveResultToOutput(repostedChannel)
|
||||
extraTxos = append(extraTxos, output)
|
||||
}
|
||||
|
||||
return txos, extraTxos, nil
|
||||
}
|
||||
|
||||
return nil, nil, nil
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) Resolve(ctx context.Context, args *pb.StringArray) (*pb.Outputs, error) {
|
||||
|
@ -521,25 +445,16 @@ func (s *Server) Resolve(ctx context.Context, args *pb.StringArray) (*pb.Outputs
|
|||
allExtraTxos := make([]*pb.Output, 0)
|
||||
|
||||
for _, url := range args.Value {
|
||||
res := db.Resolve(s.DB, url)
|
||||
txos, extraTxos, err := ExpandedResolveResultToOutput(res)
|
||||
res := s.DB.Resolve(url)
|
||||
txos, extraTxos, err := res.ToOutputs()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// FIXME: there may be a more efficient way to do this.
|
||||
// TODO: there may be a more efficient way to do this.
|
||||
allTxos = append(allTxos, txos...)
|
||||
allExtraTxos = append(allExtraTxos, extraTxos...)
|
||||
}
|
||||
|
||||
// for _, row := range allExtraTxos {
|
||||
// for _, txo := range allExtraTxos {
|
||||
// if txo.TxHash == row.TxHash && txo.Nout == row.Nout {
|
||||
// txo.Extra = row.Extra
|
||||
// break
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
|
||||
res := &pb.Outputs{
|
||||
Txos: allTxos,
|
||||
ExtraTxos: allExtraTxos,
|
||||
|
|
|
@ -219,8 +219,8 @@ func UDPPing(ip, port string) (*SPVPong, error) {
|
|||
// UDPServer is a goroutine that starts an udp server that implements the hubs
|
||||
// Ping/Pong protocol to find out about each other without making full TCP
|
||||
// connections.
|
||||
func UDPServer(args *Args) error {
|
||||
address := ":" + args.Port
|
||||
func (s *Server) UDPServer() error {
|
||||
address := ":" + s.Args.Port
|
||||
tip := make([]byte, 32)
|
||||
addr, err := net.ResolveUDPAddr("udp", address)
|
||||
if err != nil {
|
||||
|
@ -243,7 +243,7 @@ func UDPServer(args *Args) error {
|
|||
}
|
||||
|
||||
sAddr := addr.IP.String()
|
||||
pong := makeSPVPong(defaultFlags|availableFlag, 0, tip, sAddr, args.Country)
|
||||
pong := makeSPVPong(defaultFlags|availableFlag, 0, tip, sAddr, s.Args.Country)
|
||||
data := pong.Encode()
|
||||
|
||||
_, err = conn.WriteToUDP(data, addr)
|
||||
|
|
|
@ -12,6 +12,7 @@ import (
|
|||
"syscall"
|
||||
)
|
||||
|
||||
// initsignals sets the signals to be caught by the signal handler
|
||||
func initsignals() {
|
||||
interruptSignals = []os.Signal{os.Interrupt, syscall.SIGTERM}
|
||||
}
|
||||
|
|
Loading…
Add table
Reference in a new issue