You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2020/08/21 02:31:25 UTC

[GitHub] [pulsar-client-go] shohi opened a new pull request #323: define logger interface and add Logger field to ClientOptions

shohi opened a new pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323


   ### Motivation
   enable users to configure the logger used by the client and use their own implementation. If no logger is provided, a wrapped `logrus.StandardLogger()` will be used. <s>This PR only solved part of the problem mentioned in the issue https://github.com/apache/pulsar-client-go/issues/228.</s>
   
   ### Modifications
   
   * define `Logger` and `Entry` interfaces used by the client
   * add `Logger` field to ClientOptions
   * add `logger` field to internal structures
   * provide a logger implementation backed by logrus
   * implement a no-op logger
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r471217577



##########
File path: pulsar/producer_impl.go
##########
@@ -61,7 +60,7 @@ type producer struct {
 	ticker        *time.Ticker
 	tickerStop    chan struct{}
 
-	log *log.Entry

Review comment:
       as mentioned above, i think using `logger` as field name is more idiomatic.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh removed a comment on pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh removed a comment on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-666133143






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705919799


   @shohi Can you help merge master code and fix conflict? 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-677894896


   @shohi , please take a look at #354 
   
   1. Reduced interface scope to single `Logger`
   2. Allow `logrus.FieldLogger` as `NewLoggerWithLogrus` argument (this will allow passing logrus entries as new Logger so user can populate needed fields)
   3. Use `log` instead of `logger` field name.
   
   IMO we can rename `log` to `logger` in separate PR to reduce merge conflicts and PR scope.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r464740422



##########
File path: pulsar/log/log.go
##########
@@ -0,0 +1,51 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// DefaultNopLogger returns a nop logger.
+func DefaultNopLogger() Logger {
+	return nopLogger{}
+}
+
+type nopLogger struct{}
+
+func (l nopLogger) SubLogger(fields Fields) Logger                 { return l }
+func (l nopLogger) WithFields(fields Fields) Entry                 { return nopEntry{} }
+func (l nopLogger) WithField(name string, value interface{}) Entry { return nopEntry{} }
+func (l nopLogger) Debug(args ...interface{})                      {}
+func (l nopLogger) Info(args ...interface{})                       {}
+func (l nopLogger) Warn(args ...interface{})                       {}
+func (l nopLogger) Error(args ...interface{})                      {}
+func (l nopLogger) Debugf(format string, args ...interface{})      {}
+func (l nopLogger) Infof(format string, args ...interface{})       {}
+func (l nopLogger) Warnf(format string, args ...interface{})       {}
+func (l nopLogger) Errorf(format string, args ...interface{})      {}
+

Review comment:
       To make the interfaces concise and small, I don't think exiting and panicing are in the scope of a logging library. (inspired by [logur/logur](https://github.com/logur/logur/blob/master/logger.go#L39) 

##########
File path: pulsar/log/log.go
##########
@@ -0,0 +1,51 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// DefaultNopLogger returns a nop logger.
+func DefaultNopLogger() Logger {
+	return nopLogger{}
+}
+
+type nopLogger struct{}
+
+func (l nopLogger) SubLogger(fields Fields) Logger                 { return l }
+func (l nopLogger) WithFields(fields Fields) Entry                 { return nopEntry{} }
+func (l nopLogger) WithField(name string, value interface{}) Entry { return nopEntry{} }
+func (l nopLogger) Debug(args ...interface{})                      {}
+func (l nopLogger) Info(args ...interface{})                       {}
+func (l nopLogger) Warn(args ...interface{})                       {}
+func (l nopLogger) Error(args ...interface{})                      {}
+func (l nopLogger) Debugf(format string, args ...interface{})      {}
+func (l nopLogger) Infof(format string, args ...interface{})       {}
+func (l nopLogger) Warnf(format string, args ...interface{})       {}
+func (l nopLogger) Errorf(format string, args ...interface{})      {}
+

Review comment:
       To make the interfaces concise and small, I don't think exiting and panicing are in the scope of a logging library. (inspired by [logur/logur](https://github.com/logur/logur/blob/master/logger.go#L39))




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh commented on pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-666133143






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-657927354


   @shohi It seems that you and #322  are solving the same thing, right?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi removed a comment on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi removed a comment on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-678003212


   > @shohi , please take a look at #354
   > 
   >     1. Reduced interface scope to single `Logger`
   > 
   >     2. Allow `logrus.FieldLogger` as `NewLoggerWithLogrus` argument (this will allow passing logrus entries as new Logger so user can populate needed fields)
   > 
   >     3. Use `log` instead of `logger` field name.
   > 
   > 
   > IMO we can rename `log` to `logger` in separate PR to reduce merge conflicts and PR scope.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705933959


   @wolfstudy the codes have been updated, but i don't have the write access to the repo, could you please help merge the PR?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-678004467


   > @shohi , please take a look at #354
   > 
   >     1. Reduced interface scope to single `Logger`
   > 
   >     2. Allow `logrus.FieldLogger` as `NewLoggerWithLogrus` argument (this will allow passing logrus entries as new Logger so user can populate needed fields)
   > 
   >     3. Use `log` instead of `logger` field name.
   > 
   > 
   > IMO we can rename `log` to `logger` in separate PR to reduce merge conflicts and PR scope.
   
   I have some concerns about point 1 - see my comments above, and agree with the others.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r470986568



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry
+
+	Debug(args ...interface{})
+	Info(args ...interface{})
+	Warn(args ...interface{})
+	Error(args ...interface{})
+
+	Debugf(format string, args ...interface{})
+	Infof(format string, args ...interface{})
+	Warnf(format string, args ...interface{})
+	Errorf(format string, args ...interface{})
+}
+
+// Entry describes the interface for the logger entry.
+type Entry interface {

Review comment:
       Is it possible to leave one interface? E.g. `SubLogger(fields Fields) Logger` is semantically close to  `WithFields(fields Fields) Entry`, so probably we can leave just `Entry`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474355182



##########
File path: pulsar/producer_partition.go
##########
@@ -89,7 +88,7 @@ type partitionProducer struct {
 	state  int32
 	client *client
 	topic  string
-	log    *log.Entry
+	logger log.Logger

Review comment:
       Yeah, renaming makes the review scope bigger, but i don't think it's hard to review, it's just a little annoying. The field name using `log` really confuses me, perhaps it's better to open another PR for naming refactor, how about putting a TODO marker first?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-680422646


   Sure, will update it


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474242883



##########
File path: pulsar/producer_partition.go
##########
@@ -89,7 +88,7 @@ type partitionProducer struct {
 	state  int32
 	client *client
 	topic  string
-	log    *log.Entry
+	logger log.Logger

Review comment:
       It's true, but it makes review scope unfortunately bigger :(




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy merged pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy merged pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh removed a comment on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh removed a comment on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705921803


   > @shohi Can you help merge master code and fix conflict?
   
   Sure, i'll apply the code change from the master branch, but not sure I have the permission to merge codes


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi removed a comment on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi removed a comment on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-680422646


   Sure, will update it


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi edited a comment on pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi edited a comment on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-657936365


   @wolfstudy Not the same approach. This PR enables users to pass a customized logger when creating clients, while https://github.com/apache/pulsar-client-go/pull/322 adds methods to alter the global logger.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r462692432



##########
File path: pulsar/client.go
##########
@@ -91,6 +92,9 @@ type ClientOptions struct {
 
 	// Max number of connections to a single broker that will kept in the pool. (Default: 1 connection)
 	MaxConnectionsPerBroker int
+
+	// Configure the logger used by the client. (Default: logrus.StandardLogger())
+	Logger *logrus.Logger

Review comment:
       Agreed, will add logger and entry interface later.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705922564


   @wolfstudy Sure, i'll apply the code change from the master branch, but not sure i have the permission to merge codes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] snowcrumble commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
snowcrumble commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r464314524



##########
File path: pulsar/log/wrapper_logrus.go
##########
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+import (
+	"github.com/sirupsen/logrus"
+)
+
+// logrusWrapper implements Logger interface
+// based on underlying logrus.FieldLogger
+type logrusWrapper struct {
+	l logrus.FieldLogger
+}

Review comment:
       If methods set of interface Logger is a sub set of `*logrus.Logger`, you can directly new a `logrus.Logger` as interface `Logger` to be the default logger without define this wrapper.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474245317



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry
+
+	Debug(args ...interface{})
+	Info(args ...interface{})
+	Warn(args ...interface{})
+	Error(args ...interface{})
+
+	Debugf(format string, args ...interface{})
+	Infof(format string, args ...interface{})
+	Warnf(format string, args ...interface{})
+	Errorf(format string, args ...interface{})
+}
+
+// Entry describes the interface for the logger entry.
+type Entry interface {

Review comment:
       Currently there is no pooling, so having two entities just complicates implementation and slightly confuses.
   The `SubLogger(fields Fields)` result is technically `logrus.Entry`. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] snowcrumble commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
snowcrumble commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r464314524



##########
File path: pulsar/log/wrapper_logrus.go
##########
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+import (
+	"github.com/sirupsen/logrus"
+)
+
+// logrusWrapper implements Logger interface
+// based on underlying logrus.FieldLogger
+type logrusWrapper struct {
+	l logrus.FieldLogger
+}

Review comment:
       If interface Logger's method is a sub set of `*logrus.Logger`, you can directly new a `logrus.Logger` as interface `Logger` to be the default logger without define this wrapper.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r471198778



##########
File path: pulsar/producer_partition.go
##########
@@ -89,7 +88,7 @@ type partitionProducer struct {
 	state  int32
 	client *client
 	topic  string
-	log    *log.Entry
+	logger log.Logger

Review comment:
       `log` is usually used for the package name, the logger is the one who does the logging job, i think it's more accurate using `logger` than `log`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-680422495


   @shohi Can you help provide the docs in `docs` folder? How users can use this feature to customize their logs.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r470983748



##########
File path: pulsar/producer_impl.go
##########
@@ -61,7 +60,7 @@ type producer struct {
 	ticker        *time.Ticker
 	tickerStop    chan struct{}
 
-	log *log.Entry

Review comment:
       Same excessive field rename. Can you please reduce review scope? ^^ 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r471202834



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry
+
+	Debug(args ...interface{})
+	Info(args ...interface{})
+	Warn(args ...interface{})
+	Error(args ...interface{})
+
+	Debugf(format string, args ...interface{})
+	Infof(format string, args ...interface{})
+	Warnf(format string, args ...interface{})
+	Errorf(format string, args ...interface{})
+}
+
+// Entry describes the interface for the logger entry.
+type Entry interface {

Review comment:
       Yeah, `Logger` and `Entry` share many methods, but their responsibilities are different. In general, `Entry` just represents one log entry, once fired it should be released. Separating entry from logger enables users to fine-grained control the implementation, such as pooling the entries. I'm not sure whether it's more friendly and easier to use by combining the two interfaces into one.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r464740422



##########
File path: pulsar/log/log.go
##########
@@ -0,0 +1,51 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// DefaultNopLogger returns a nop logger.
+func DefaultNopLogger() Logger {
+	return nopLogger{}
+}
+
+type nopLogger struct{}
+
+func (l nopLogger) SubLogger(fields Fields) Logger                 { return l }
+func (l nopLogger) WithFields(fields Fields) Entry                 { return nopEntry{} }
+func (l nopLogger) WithField(name string, value interface{}) Entry { return nopEntry{} }
+func (l nopLogger) Debug(args ...interface{})                      {}
+func (l nopLogger) Info(args ...interface{})                       {}
+func (l nopLogger) Warn(args ...interface{})                       {}
+func (l nopLogger) Error(args ...interface{})                      {}
+func (l nopLogger) Debugf(format string, args ...interface{})      {}
+func (l nopLogger) Infof(format string, args ...interface{})       {}
+func (l nopLogger) Warnf(format string, args ...interface{})       {}
+func (l nopLogger) Errorf(format string, args ...interface{})      {}
+

Review comment:
       To make the interfaces concise and small, I think exiting and panicing are out of the scope of a logging library. (inspired by [logur/logur](https://github.com/logur/logur/blob/master/logger.go#L39))




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi closed pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi closed pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474465298



##########
File path: pulsar/producer_partition.go
##########
@@ -89,7 +88,7 @@ type partitionProducer struct {
 	state  int32
 	client *client
 	topic  string
-	log    *log.Entry
+	logger log.Logger

Review comment:
       TODO marker is ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r471211624



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry

Review comment:
       Make sense, it will be handy and less error-prone if the `WithError` is available for pulsar-client-go, although the logging features are already complete.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-657936365


   @wolfstudy Not the same approach. This PR enables users to pass a given logger when creating clients, while https://github.com/apache/pulsar-client-go/pull/322 adds methods to alter the global logger.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705922564






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474373341



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry
+
+	Debug(args ...interface{})
+	Info(args ...interface{})
+	Warn(args ...interface{})
+	Error(args ...interface{})
+
+	Debugf(format string, args ...interface{})
+	Infof(format string, args ...interface{})
+	Warnf(format string, args ...interface{})
+	Errorf(format string, args ...interface{})
+}
+
+// Entry describes the interface for the logger entry.
+type Entry interface {

Review comment:
       For `logrus` backed implementation, it's true, but the logger interfaces are not targeted only for logrus, they are more for users to customize the implementation to cope with their own scenarios. 
   
   Using two entities makes logging concepts clear and gives users more control on the implementation, in some cases,  logging performance is critical and entry pooling is necessary. Logger is usually used to hold the log level, common fields and its own entry pool, where the entry holds its private fields, formats all its fields into bytes and emits the bytes out.  I think implementation based on these concepts is much straightforward. The popular `logrus` provides both logger-level and entry-level methods, but they are not based on interfaces. Another the performant logging library `zap` provides only Logger level methods, but all loggers share a global entry pool. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r464742180



##########
File path: pulsar/log/wrapper_logrus.go
##########
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+import (
+	"github.com/sirupsen/logrus"
+)
+
+// logrusWrapper implements Logger interface
+// based on underlying logrus.FieldLogger
+type logrusWrapper struct {
+	l logrus.FieldLogger
+}

Review comment:
       thanks for your suggestion :-). Actually, the method set of interfaces is not a subset of logrus.Logger
   *  the logger/entry interfaces defined in pulsar-client-go are independent of any existing logger implementations.
   * the logger has its own `Fields` type even though it's similar to `logrus`'s, but they are not the same.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] merlimat commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r459822291



##########
File path: pulsar/client.go
##########
@@ -91,6 +92,9 @@ type ClientOptions struct {
 
 	// Max number of connections to a single broker that will kept in the pool. (Default: 1 connection)
 	MaxConnectionsPerBroker int
+
+	// Configure the logger used by the client. (Default: logrus.StandardLogger())
+	Logger *logrus.Logger

Review comment:
       My main concern is that once we expose in API, people can depend on it. So we should rather first define the logger API and then apply this change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r470982958



##########
File path: pulsar/producer_partition.go
##########
@@ -89,7 +88,7 @@ type partitionProducer struct {
 	state  int32
 	client *client
 	topic  string
-	log    *log.Entry
+	logger log.Logger

Review comment:
       Looks like excessive rename. `log` field is OK too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r454753639



##########
File path: pulsar/client.go
##########
@@ -91,6 +92,9 @@ type ClientOptions struct {
 
 	// Max number of connections to a single broker that will kept in the pool. (Default: 1 connection)
 	MaxConnectionsPerBroker int
+
+	// Configure the logger used by the client. (Default: logrus.StandardLogger())
+	Logger *logrus.Logger

Review comment:
       Yeah, i agree with you. It may need some effort to extract a reasonable logger interface, how about resolving it in another PR?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474249200



##########
File path: pulsar/producer_partition.go
##########
@@ -89,7 +88,7 @@ type partitionProducer struct {
 	state  int32
 	client *client
 	topic  string
-	log    *log.Entry
+	logger log.Logger

Review comment:
       `log` were selected initially by package maintainers, IMO there is no reason to change naming in scope of this PR.
   It is really hard to review.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-680418259


   /cc @wolfstudy could the team provide some feedback about this PR? can it be merged?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r462692432



##########
File path: pulsar/client.go
##########
@@ -91,6 +92,9 @@ type ClientOptions struct {
 
 	// Max number of connections to a single broker that will kept in the pool. (Default: 1 connection)
 	MaxConnectionsPerBroker int
+
+	// Configure the logger used by the client. (Default: logrus.StandardLogger())
+	Logger *logrus.Logger

Review comment:
       Agreed, will add logger and entry interface later.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] merlimat commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r454487155



##########
File path: pulsar/client.go
##########
@@ -91,6 +92,9 @@ type ClientOptions struct {
 
 	// Max number of connections to a single broker that will kept in the pool. (Default: 1 connection)
 	MaxConnectionsPerBroker int
+
+	// Configure the logger used by the client. (Default: logrus.StandardLogger())
+	Logger *logrus.Logger

Review comment:
       I like this approach, though in this case we'd be exposing Logrus as part of our API, something that we cannot later rollback. 
   
   If we want to take a logger option in, we should have a custom logger interface (with default impl bound to logrus).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy merged pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy merged pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474474194



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry
+
+	Debug(args ...interface{})
+	Info(args ...interface{})
+	Warn(args ...interface{})
+	Error(args ...interface{})
+
+	Debugf(format string, args ...interface{})
+	Infof(format string, args ...interface{})
+	Warnf(format string, args ...interface{})
+	Errorf(format string, args ...interface{})
+}
+
+// Entry describes the interface for the logger entry.
+type Entry interface {

Review comment:
       Agree, but it seems like logrus approach with fields (`map[string]interface{}`) will reduce benefits from pooling.
   
   Moreover, attempt to abstract logging is already limits performance optimizations due to usage of interfaces instead of structures, so I'm not sure that having both interfaces will simplify something performance-related.
   
   There is another point: we are pre-v1, so it is OK to break backward compatibility at some point. If one decide to switch to zap or zerolog and employ it's performance benefits, they will be forced to change interfaces (but I still think that it is nearly impossible to create both abstracted and performant logger interfaces) 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r462992494



##########
File path: pulsar/client.go
##########
@@ -91,6 +92,9 @@ type ClientOptions struct {
 
 	// Max number of connections to a single broker that will kept in the pool. (Default: 1 connection)
 	MaxConnectionsPerBroker int
+
+	// Configure the logger used by the client. (Default: logrus.StandardLogger())
+	Logger *logrus.Logger

Review comment:
       it makes sense, i defined the `Logger` and `Entry` interfaces at `pulsar/log` package, please take a look :-)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r471198778



##########
File path: pulsar/producer_partition.go
##########
@@ -89,7 +88,7 @@ type partitionProducer struct {
 	state  int32
 	client *client
 	topic  string
-	log    *log.Entry
+	logger log.Logger

Review comment:
       `log` is usually used for the package name, the logger is the one who does the logging job, i think it's more accurate using `logger` than `log`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-680434590


   The logger interfaces are much straightforward - almost same approach applied by logrus or zap, users can refer these implementations for some hints.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r464742180



##########
File path: pulsar/log/wrapper_logrus.go
##########
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+import (
+	"github.com/sirupsen/logrus"
+)
+
+// logrusWrapper implements Logger interface
+// based on underlying logrus.FieldLogger
+type logrusWrapper struct {
+	l logrus.FieldLogger
+}

Review comment:
       thanks for your suggestion :-). Actually, the method set of interfaces is not a subset of logrus.Logger
   *  the logger/entry interfaces defined in pulsar-client-go are independent of any existing logger implementations.
   * the logger has its own `Fields` type even though it's similar with `logrus`, but they are not the same.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705921803


   > @shohi Can you help merge master code and fix conflict?
   
   Sure, i'll apply the code change from the master branch, but not sure I have the permission to merge codes


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-678003212


   > @shohi , please take a look at #354
   > 
   >     1. Reduced interface scope to single `Logger`
   > 
   >     2. Allow `logrus.FieldLogger` as `NewLoggerWithLogrus` argument (this will allow passing logrus entries as new Logger so user can populate needed fields)
   > 
   >     3. Use `log` instead of `logger` field name.
   > 
   > 
   > IMO we can rename `log` to `logger` in separate PR to reduce merge conflicts and PR scope.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705941345


   Thanks @shohi work for this


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r471211624



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry

Review comment:
       Make sense, it will be handy and less error-prone if the `WithError` is available for pulsar-client-go, although the logging features are already complete. :-)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] snowcrumble commented on a change in pull request #323: add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
snowcrumble commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r464311568



##########
File path: pulsar/log/log.go
##########
@@ -0,0 +1,51 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// DefaultNopLogger returns a nop logger.
+func DefaultNopLogger() Logger {
+	return nopLogger{}
+}
+
+type nopLogger struct{}
+
+func (l nopLogger) SubLogger(fields Fields) Logger                 { return l }
+func (l nopLogger) WithFields(fields Fields) Entry                 { return nopEntry{} }
+func (l nopLogger) WithField(name string, value interface{}) Entry { return nopEntry{} }
+func (l nopLogger) Debug(args ...interface{})                      {}
+func (l nopLogger) Info(args ...interface{})                       {}
+func (l nopLogger) Warn(args ...interface{})                       {}
+func (l nopLogger) Error(args ...interface{})                      {}
+func (l nopLogger) Debugf(format string, args ...interface{})      {}
+func (l nopLogger) Infof(format string, args ...interface{})       {}
+func (l nopLogger) Warnf(format string, args ...interface{})       {}
+func (l nopLogger) Errorf(format string, args ...interface{})      {}
+

Review comment:
       why no Panic and Fatal?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r476988275



##########
File path: pulsar/internal/connection.go
##########
@@ -188,21 +188,30 @@ type connection struct {
 	maxMessageSize int32
 }
 
-func newConnection(logicalAddr *url.URL, physicalAddr *url.URL, tlsOptions *TLSOptions,
-	connectionTimeout time.Duration, auth auth.Provider) *connection {
+// ConnectionOptions defines configurations for creating connection.
+type ConnectionOptions struct {
+	LogicalAddr       *url.URL
+	PhysicalAddr      *url.URL
+	TLS               *TLSOptions
+	ConnectionTimeout time.Duration
+	Auth              auth.Provider
+	Logger            log.Logger
+}

Review comment:
       If `ConnectionOptions` is only used as a parameter of `newConnection()`, can we consider it as a private?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r476991887



##########
File path: pulsar/internal/connection.go
##########
@@ -188,21 +188,30 @@ type connection struct {
 	maxMessageSize int32
 }
 
-func newConnection(logicalAddr *url.URL, physicalAddr *url.URL, tlsOptions *TLSOptions,
-	connectionTimeout time.Duration, auth auth.Provider) *connection {
+// ConnectionOptions defines configurations for creating connection.
+type ConnectionOptions struct {
+	LogicalAddr       *url.URL
+	PhysicalAddr      *url.URL
+	TLS               *TLSOptions
+	ConnectionTimeout time.Duration
+	Auth              auth.Provider
+	Logger            log.Logger
+}

Review comment:
       Sure, will update it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r470982015



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry

Review comment:
       IMO there should be also `WithError(err) Entry` method, `WithField("cause", err)`  is not 1:1 replacement and more error-prone (e.g. somebody can make a typo field name).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh removed a comment on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh removed a comment on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705921803


   > @shohi Can you help merge master code and fix conflict?
   
   Sure, i'll apply the code change from the master branch, but not sure I have the permission to merge codes


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wsoh commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wsoh commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705921803


   > @shohi Can you help merge master code and fix conflict?
   
   Sure, i'll apply the code change from the master branch, but not sure I have the permission to merge codes


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] ernado commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
ernado commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474265634



##########
File path: pulsar/internal/commands.go
##########
@@ -196,20 +194,21 @@ func baseCommand(cmdType pb.BaseCommand_Type, msg proto.Message) *pb.BaseCommand
 	case pb.BaseCommand_AUTH_RESPONSE:
 		cmd.AuthResponse = msg.(*pb.CommandAuthResponse)
 	default:
-		log.Panic("Missing command type: ", cmdType)
+		panic(fmt.Sprintf("Missing command type: %v", cmdType))
 	}
 
 	return cmd
 }
 
-func addSingleMessageToBatch(wb Buffer, smm *pb.SingleMessageMetadata, payload []byte) {
+func addSingleMessageToBatch(wb Buffer, smm *pb.SingleMessageMetadata, payload []byte, logger log.Logger) {
 	metadataSize := uint32(smm.Size())
 	wb.WriteUint32(metadataSize)
 
 	wb.ResizeIfNeeded(metadataSize)
 	_, err := smm.MarshalToSizedBuffer(wb.WritableSlice()[:metadataSize])
 	if err != nil {
-		log.WithError(err).Fatal("Protobuf serialization error")
+		logger.WithError(err).Error("Protobuf serialization error")

Review comment:
       Can we leave `Fatal` here? It is different from Error + panic.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#issuecomment-705919799






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] shohi commented on a change in pull request #323: define logger interface and add Logger field to ClientOptions

Posted by GitBox <gi...@apache.org>.
shohi commented on a change in pull request #323:
URL: https://github.com/apache/pulsar-client-go/pull/323#discussion_r474600138



##########
File path: pulsar/log/logger.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package log
+
+// Logger and Entry interfaces are inspired by sirupsen/logrus
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Logger describes the interface that must be implemeted by all loggers.
+type Logger interface {
+	SubLogger(fields Fields) Logger
+
+	WithFields(fields Fields) Entry
+	WithField(name string, value interface{}) Entry
+
+	Debug(args ...interface{})
+	Info(args ...interface{})
+	Warn(args ...interface{})
+	Error(args ...interface{})
+
+	Debugf(format string, args ...interface{})
+	Infof(format string, args ...interface{})
+	Warnf(format string, args ...interface{})
+	Errorf(format string, args ...interface{})
+}
+
+// Entry describes the interface for the logger entry.
+type Entry interface {

Review comment:
       Yeah, usage of interfaces and map-based field will cause some performance loss, but usually effective pooling and avoiding reflection during formatting are more important for performance, and sometimes we need to trade off flexibility against performance.
   
   Both the two entities are essential for logger, or as interfaces  or as structs. Using two interfaces provides a guide for implementation, which also implies the implementer can pool entry to improve performance if needed. If only Logger interface is provided, the underlying created entry needs to comply the logger interface, otherwise it's not clear how the `WithField` works. Besides,`SubLogger` can initialize the common fields for successive log entries, where `WithFields` enables each entry to add its private fields. I think it's hard to handle this case If only one interface is available. So, IMO it's better to expose the Entry interface.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org