|
|
@@ -1353,11 +1353,12 @@ func (c *Conn) query(ctx context.Context, statement string, values ...interface{
|
|
|
|
|
|
func (c *Conn) awaitSchemaAgreement(ctx context.Context) (err error) {
|
|
|
const (
|
|
|
- peerSchemas = "SELECT schema_version, peer FROM system.peers"
|
|
|
+ peerSchemas = "SELECT * FROM system.peers"
|
|
|
localSchemas = "SELECT schema_version FROM system.local WHERE key='local'"
|
|
|
)
|
|
|
|
|
|
var versions map[string]struct{}
|
|
|
+ var schemaVersion string
|
|
|
|
|
|
endDeadline := time.Now().Add(c.session.cfg.MaxWaitSchemaAgreement)
|
|
|
for time.Now().Before(endDeadline) {
|
|
|
@@ -1365,16 +1366,22 @@ func (c *Conn) awaitSchemaAgreement(ctx context.Context) (err error) {
|
|
|
|
|
|
versions = make(map[string]struct{})
|
|
|
|
|
|
- var schemaVersion string
|
|
|
- var peer string
|
|
|
- for iter.Scan(&schemaVersion, &peer) {
|
|
|
- if schemaVersion == "" {
|
|
|
- Logger.Printf("skipping peer entry with empty schema_version: peer=%q", peer)
|
|
|
+ rows, err := iter.SliceMap()
|
|
|
+ if err != nil {
|
|
|
+ goto cont
|
|
|
+ }
|
|
|
+
|
|
|
+ for _, row := range rows {
|
|
|
+ host, err := c.session.hostInfoFromMap(row, c.session.cfg.Port)
|
|
|
+ if err != nil {
|
|
|
+ goto cont
|
|
|
+ }
|
|
|
+ if !isValidPeer(host) || host.schemaVersion == "" {
|
|
|
+ Logger.Printf("invalid peer or peer with empty schema_version: peer=%q", host)
|
|
|
continue
|
|
|
}
|
|
|
|
|
|
- versions[schemaVersion] = struct{}{}
|
|
|
- schemaVersion = ""
|
|
|
+ versions[host.schemaVersion] = struct{}{}
|
|
|
}
|
|
|
|
|
|
if err = iter.Close(); err != nil {
|